You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Ying Zheng <yi...@uber.com.INVALID> on 2020/08/04 04:57:42 UTC

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

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 <ju...@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 <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
> > unclean-leader-election.
> > 2) Since the user set local retention time (or local retention bytes), I
> > think we are expected to
> > keep that much local data when possible (avoid truncating all the local
> > data). But, as you said,
> > unclean leader elections are very rare, this may not be a big problem.
> >
> > The current design uses the leader broker as source-of-truth. This is
> > consistent with the
> > existing Kafka behavior.
> >
> > By using remote storage as the source-of-truth, the follower logic can
> be a
> > little simpler,
> > but the leader logic is going to be more complex. Overall, I don't see
> > there many benefits
> > of using remote storage as the source-of-truth.
> >
> >
> >
> > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Satish,
> > >
> > > Thanks for the reply.
> > >
> > > 1001. In your example, I was thinking that you could just download the
> > > latest leader epoch from the object store. After that you know the
> leader
> > > should end with offset 1100. The leader will delete all its local data
> > > before offset 1000 and start accepting new messages at offset 1100.
> > > Consumer requests for messages before offset 1100 will be served from
> the
> > > object store. The benefit with this approach is that it's simpler to
> > reason
> > > about who is the source of truth. The downside is slightly  increased
> > > unavailability window during unclean leader election. Since unclean
> > leader
> > > elections are rare, I am not sure if this is a big concern.
> > >
> > > 1008. Yes, I think introducing sth like local.retention.ms seems more
> > > consistent.
> > >
> > > Jun
> > >
> > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> satish.duggana@gmail.com
> > >
> > > wrote:
> > >
> > > > HI Jun,
> > > > Thanks for your comments. We put our inline replies below.
> > > >
> > > > 1001. I was thinking that you could just use the tiered metadata to
> do
> > > the
> > > > reconciliation. The tiered metadata contains offset ranges and epoch
> > > > history. Those should be enough for reconciliation purposes.
> > > >
> > > > If we use remote storage as the source-of-truth during
> > > > unclean-leader-election, it's possible that after reconciliation the
> > > > remote storage will have more recent data than the new leader's local
> > > > storage. For example, the new leader's latest message is offset 1000,
> > > > while the remote storage has message 1100. In such a case, the new
> > > > leader will have to download the messages from 1001 to 1100, before
> > > > accepting new messages from producers. Otherwise, there would be a
> gap
> > > > in the local data between 1000 and 1101.
> > > >
> > > > Moreover, with the current design, leader epoch history is stored in
> > > > remote storage, rather than the metadata topic. We did consider
> saving
> > > > epoch history in remote segment metadata. But the concern is that
> > > > there is currently no limit for the epoch history size.
> Theoretically,
> > > > if a user has a very long remote retention time and there are very
> > > > frequent leadership changes, the leader epoch history can become too
> > > > long to fit into a regular Kafka message.
> > > >
> > > >
> > > > 1003.3 Having just a serverEndpoint string is probably not enough.
> > > > Connecting to a Kafka cluster may need various security credentials.
> We
> > > can
> > > > make RLMM configurable and pass in the properties through the
> > configure()
> > > > method. Ditto for RSM.
> > > >
> > > > RLMM and  RSM are already configurable and they take properties which
> > > > start with "remote.log.metadata." and "remote.log.storage."
> > > > respectively and a few others. We have listener-name as the config
> for
> > > > RLMM and other properties(like security) can be sent as you
> suggested.
> > > > We will update the KIP with the details.
> > > >
> > > >
> > > > 1008.1 We started with log.retention.hours and log.retention.minutes,
> > and
> > > > added log.retention.ms later. If we are adding a new configuration,
> ms
> > > > level config alone is enough and is simpler. We can build tools to
> make
> > > the
> > > > configuration at different granularities easier. The definition of
> > > > log.retention.ms is "The number of milliseconds to keep a log file
> > > before
> > > > deleting it". The deletion is independent of whether tiering is
> enabled
> > > or
> > > > not. If this changes to just the local portion of the data, we are
> > > changing
> > > > the meaning of an existing configuration.
> > > >
> > > > We are fine with either way. We can go with log.retention.xxxx as the
> > > > effective log retention instead of local log retention. With this
> > > > convention, we need to introduce  local.log.retention instead of
> > > > remote.log.retention.ms that we proposed. If log.retention.ms as -1
> > > > then remote retention is also considered as unlimited but user should
> > > > be able to set the local.retention.ms.
> > > > So, we need to introduce local.log.retention.ms and
> > > > local.log.retention.bytes which should  always  be <=
> > > > log.retention.ms/bytes respectively.
> > > >
> > > >
> > > >
> > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the reply. A few quick comments below.
> > > > >
> > > > > 1001. I was thinking that you could just use the tiered metadata to
> > do
> > > > the
> > > > > reconciliation. The tiered metadata contains offset ranges and
> epoch
> > > > > history. Those should be enough for reconciliation purposes.
> > > > >
> > > > > 1003.3 Having just a serverEndpoint string is probably not enough.
> > > > > Connecting to a Kafka cluster may need various security
> credentials.
> > We
> > > > can
> > > > > make RLMM configurable and pass in the properties through the
> > > configure()
> > > > > method. Ditto for RSM.
> > > > >
> > > > > 1008.1 We started with log.retention.hours and
> log.retention.minutes,
> > > and
> > > > > added log.retention.ms later. If we are adding a new
> configuration,
> > ms
> > > > > level config alone is enough and is simpler. We can build tools to
> > make
> > > > the
> > > > > configuration at different granularities easier. The definition of
> > > > > log.retention.ms is "The number of milliseconds to keep a log file
> > > > before
> > > > > deleting it". The deletion is independent of whether tiering is
> > enabled
> > > > or
> > > > > not. If this changes to just the local portion of the data, we are
> > > > changing
> > > > > the meaning of an existing configuration.
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > >
> > > > > > Thank you for the comments! Ying, Harsha and I discussed and put
> > our
> > > > > > comments below.
> > > > > >
> > > > > >
> > > > > > 1001. The KIP described a few scenarios of unclean leader
> > elections.
> > > > This
> > > > > > is very useful, but I am wondering if this is the best approach.
> My
> > > > > > understanding of the proposed approach is to allow the new
> > (unclean)
> > > > leader
> > > > > > to take new messages immediately. While this increases
> > availability,
> > > it
> > > > > > creates the problem that there could be multiple conflicting
> > segments
> > > > in
> > > > > > the remote store for the same offset range. This seems to make it
> > > > harder
> > > > > > for RLMM to determine which archived log segments contain the
> > correct
> > > > data.
> > > > > > For example, an archived log segment could at one time be the
> > correct
> > > > data,
> > > > > > but be changed to incorrect data after an unclean leader
> election.
> > An
> > > > > > alternative approach is to let the unclean leader use the
> archived
> > > > data as
> > > > > > the source of truth. So, when the new (unclean) leader takes
> over,
> > it
> > > > first
> > > > > > reconciles the local data based on the archived data before
> taking
> > > new
> > > > > > messages. This makes the job of RLMM a bit easier since all
> > archived
> > > > data
> > > > > > are considered correct. This increases availability a bit.
> However,
> > > > since
> > > > > > unclean leader elections are rare, this may be ok.
> > > > > >
> > > > > > Firstly, We don't want to assume the remote storage is more
> > reliable
> > > > than
> > > > > > Kafka. Kafka unclean leader election usually happens when there
> is
> > a
> > > > large
> > > > > > scale outage that impacts multiple racks (or even multiple
> > > availability
> > > > > > zones). In such a case, the remote storage may be unavailable or
> > > > unstable.
> > > > > > Pulling a large amount of data from the remote storage to
> reconcile
> > > the
> > > > > > local data may also exacerbate the outage. With the current
> design,
> > > > the new
> > > > > > leader can start working even when the remote storage is
> > temporarily
> > > > > > unavailable.
> > > > > >
> > > > > > Secondly, it is not easier to implement the reconciling logic at
> > the
> > > > leader
> > > > > > side. It can take a long time for the new leader to download the
> > > remote
> > > > > > data and rebuild local producer id / leader epoch information.
> > During
> > > > this
> > > > > > period, the leader cannot accept any requests from the clients
> and
> > > > > > followers. We have to introduce a new state for the leader, and a
> > new
> > > > error
> > > > > > code to let the clients / followers know what is happening.
> > > > > >
> > > > > >
> > > > > >
> > > > > > 1002. RemoteStorageManager.
> > > > > > 1002.1 There seems to be some inconsistencies in
> > > RemoteStorageManager.
> > > > We
> > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all other
> methods,
> > > we
> > > > pass
> > > > > > in RemoteLogSegmentMetadata.
> > > > > >
> > > > > > Nice catch, we can have the RemoteLogSegmentMetadata for
> > > copyLogSegment
> > > > > > too.
> > > > > >
> > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata inclusive or
> > > exclusive?
> > > > > >
> > > > > > It is inclusive.
> > > > > >
> > > > > > 1002.3 It seems that we need an api to get the leaderEpoch
> history
> > > for
> > > > a
> > > > > > partition.
> > > > > >
> > > > > > Yes, updated the KIP with the new method.
> > > > > >
> > > > > >
> > > > > > 1002.4 Could you define the type of RemoteLogSegmentContext?
> > > > > >
> > > > > > This is removed in the latest code and it is not needed.
> > > > > >
> > > > > >
> > > > > > 1003 RemoteLogMetadataManager
> > > > > >
> > > > > > 1003.1 I am not sure why we need both of the following methods
> > > > > > in RemoteLogMetadataManager. Could we combine them into one that
> > > takes
> > > > in
> > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > >     RemoteLogSegmentId getRemoteLogSegmentId(TopicPartition
> > > > topicPartition,
> > > > > > long offset) throws IOException;
> > > > > >     RemoteLogSegmentMetadata
> > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > remoteLogSegmentId) throws IOException;
> > > > > >
> > > > > > Good point, these can be merged for now. I guess we needed them
> in
> > > > earlier
> > > > > > version of the implementation but it is not needed now.
> > > > > >
> > > > > > 1003.2 There seems to be some inconsistencies in the methods
> > below. I
> > > > am
> > > > > > not sure why one takes RemoteLogSegmentMetadata and the other
> > > > > > takes RemoteLogSegmentId.
> > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > >     void deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > remoteLogSegmentId) throws IOException;
> > > > > >
> > > > > > RLMM stores RemoteLogSegmentMetadata which is identified by
> > > > > > RemoteLogsSegmentId. So, when it is added it takes
> > > > > > RemoteLogSegmentMetadata. `delete` operation needs only
> > > > RemoteLogsSegmentId
> > > > > > as RemoteLogSegmentMetadata can be identified with
> > > RemoteLogsSegmentId.
> > > > > >
> > > > > > 1003.3 In void onServerStarted(final String serverEndpoint), what
> > > > > > is serverEndpoint used for?
> > > > > >
> > > > > > This can be used by RLMM implementation to connect to the local
> > Kafka
> > > > > > cluster. Incase of default  implementation, it is used in
> > > initializing
> > > > > > kafka clients connecting to the local cluster.
> > > > > >
> > > > > > 1004. It would be useful to document how all the new APIs are
> being
> > > > used.
> > > > > > For example, when is RemoteLogSegmentMetadata.markedForDeletion
> > being
> > > > set
> > > > > > and used? How are
> > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > used?
> > > > > >
> > > > > > RLMM APIs are going through the changes and they should be ready
> > in a
> > > > few
> > > > > > days. I will update the KIP and the mail  thread once they are
> > ready.
> > > > > >
> > > > > > 1005. Handling partition deletion: The KIP says "RLMM will
> > eventually
> > > > > > delete these segments by using RemoteStorageManager." Which
> replica
> > > > does
> > > > > > this logic?
> > > > > >
> > > > > > This is a good point. When a topic is deleted, it will not have
> any
> > > > > > leader/followers to do the cleanup. We will have a cleaner agent
> > on a
> > > > > > single broker in the cluster to do this cleanup, we plan to add
> > that
> > > in
> > > > > > controller broker.
> > > > > >
> > > > > > 1006. "If there are any failures in removing remote log segments
> > then
> > > > those
> > > > > > are stored in a specific topic (default as
> > > > __remote_segments_to_be_deleted)
> > > > > > and user can consume the events(which contain
> > remote-log-segment-id)
> > > > from
> > > > > > that topic and clean them up from remote storage.  " Not sure if
> > it's
> > > > worth
> > > > > > the complexity of adding another topic. Could we just retry?
> > > > > >
> > > > > > Sure, we can keep this simpler for now by logging an error after
> > > > retries.
> > > > > > We can give users a better way to process this in future. Oneway
> > can
> > > > be a
> > > > > > dead letter topic which can be configured by the user.
> > > > > >
> > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the existing
> > > > > > fetchPurgatory?
> > > > > >
> > > > > > We have 2 types of delayed operations waiting for 2 different
> > events.
> > > > > > DelayedFetch waits for new messages from producers.
> > > DelayedRemoteFetch
> > > > > > waits for the remote-storage-read-task to finish. When either of
> > the
> > > 2
> > > > > > events happens, we only want to notify one type of the delayed
> > > > operations.
> > > > > > It would be inefficient to put 2 types of delayed operations in
> one
> > > > > > purgatory, as the tryComplete() methods of the delayed operations
> > can
> > > > be
> > > > > > triggered by irrelevant events.
> > > > > >
> > > > > >
> > > > > > 1008. Configurations:
> > > > > > 1008.1 remote.log.retention.ms, remote.log.retention.minutes,
> > > > > > remote.log.retention.hours: It seems that we just need the ms
> one.
> > > > Also,
> > > > > > are we changing the meaning of existing config log.retention.ms
> to
> > > > mean
> > > > > > the
> > > > > > local retention? For backward compatibility, it's better to not
> > > change
> > > > the
> > > > > > meaning of existing configurations.
> > > > > >
> > > > > > We agree that we only need remote.log.retention.ms. But, the
> > > existing
> > > > > > Kafka
> > > > > > configuration
> > > > > > has 3 properties (log.retention.ms, log.retention.minutes,
> > > > > > log.retention.hours). We just
> > > > > > want to keep consistent with the existing properties.
> > > > > > Existing log.retention.xxxx config is about log retention in
> > broker’s
> > > > > > storage which is local. It should be easy for users to configure
> > > > partition
> > > > > > storage with local retention and remote retention based on their
> > > usage.
> > > > > >
> > > > > > 1008.2 Should remote.log.storage.enable be at the topic level?
> > > > > >
> > > > > > We can introduce topic level config for the same remote.log
> > settings.
> > > > User
> > > > > > can set the desired config while creating the topic.
> > > > > > remote.log.storage.enable property is not allowed to be updated
> > after
> > > > the
> > > > > > topic is created. Other remote.log.* properties can be modified.
> We
> > > > will
> > > > > > support flipping remote.log.storage.enable in next versions.
> > > > > >
> > > > > > 1009. It would be useful to list all limitations in a separate
> > > section:
> > > > > > compacted topic, JBOD, etc. Also, is changing a topic from delete
> > to
> > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > >
> > > > > > +1 to have limitations in a separate section. We will update the
> > KIP
> > > > with
> > > > > > that.
> > > > > > Topic  created with effective value for remote.log.enabled as
> true,
> > > > can not
> > > > > > change its retention policy from delete to compact.
> > > > > >
> > > > > > 1010. Thanks for performance numbers. Are those with RocksDB as
> the
> > > > cache?
> > > > > >
> > > > > > No, We have not yet added RocksDB support. This is based on
> > in-memory
> > > > map
> > > > > > representation. We will add that support and update this thread
> > after
> > > > > > updating the KIP with the numbers.
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > Satish.
> > > > > >
> > > > > >
> > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > >
> > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > >
> > > > > > > 1000. Regarding Colin's question on querying the metadata
> > directly
> > > > in the
> > > > > > > remote block store. One issue is that not all block stores
> offer
> > > the
> > > > > > needed
> > > > > > > api to query the metadata. For example, S3 only offers an api
> to
> > > list
> > > > > > > objects under a prefix and this api has the eventual
> consistency
> > > > > > semantic.
> > > > > > >
> > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > elections.
> > > > This
> > > > > > > is very useful, but I am wondering if this is the best
> approach.
> > My
> > > > > > > understanding of the proposed approach is to allow the new
> > > (unclean)
> > > > > > leader
> > > > > > > to take new messages immediately. While this increases
> > > availability,
> > > > it
> > > > > > > creates the problem that there could be multiple conflicting
> > > > segments in
> > > > > > > the remote store for the same offset range. This seems to make
> it
> > > > harder
> > > > > > > for RLMM to determine which archived log segments contain the
> > > correct
> > > > > > data.
> > > > > > > For example, an archived log segment could at one time be the
> > > correct
> > > > > > data,
> > > > > > > but be changed to incorrect data after an unclean leader
> > election.
> > > An
> > > > > > > alternative approach is to let the unclean leader use the
> > archived
> > > > data
> > > > > > as
> > > > > > > the source of truth. So, when the new (unclean) leader takes
> > over,
> > > it
> > > > > > first
> > > > > > > reconciles the local data based on the archived data before
> > taking
> > > > new
> > > > > > > messages. This makes the job of RLMM a bit easier since all
> > > archived
> > > > data
> > > > > > > are considered correct. This increases availability a bit.
> > However,
> > > > since
> > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > >
> > > > > > > 1002. RemoteStorageManager.
> > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > RemoteStorageManager. We
> > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all other
> > methods,
> > > > we
> > > > > > pass
> > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata inclusive or
> > > > exclusive?
> > > > > > > 1002.3 It seems that we need an api to get the leaderEpoch
> > history
> > > > for a
> > > > > > > partition.
> > > > > > > 1002.4 Could you define the type of RemoteLogSegmentContext?
> > > > > > >
> > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > 1003.1 I am not sure why we need both of the following methods
> > > > > > > in RemoteLogMetadataManager. Could we combine them into one
> that
> > > > takes in
> > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > >     RemoteLogSegmentId getRemoteLogSegmentId(TopicPartition
> > > > > > topicPartition,
> > > > > > > long offset) throws IOException;
> > > > > > >     RemoteLogSegmentMetadata
> > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > 1003.2 There seems to be some inconsistencies in the methods
> > below.
> > > > I am
> > > > > > > not sure why one takes RemoteLogSegmentMetadata and the other
> > > > > > > takes RemoteLogSegmentId.
> > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > >     void deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > 1003.3 In void onServerStarted(final String serverEndpoint),
> what
> > > > > > > is serverEndpoint used for?
> > > > > > >
> > > > > > > 1004. It would be useful to document how all the new APIs are
> > being
> > > > used.
> > > > > > > For example, when is RemoteLogSegmentMetadata.markedForDeletion
> > > > being set
> > > > > > > and used? How are
> > > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> being
> > > > used?
> > > > > > >
> > > > > > > 1005. Handling partition deletion: The KIP says "RLMM will
> > > eventually
> > > > > > > delete these segments by using RemoteStorageManager." Which
> > replica
> > > > does
> > > > > > > this logic?
> > > > > > >
> > > > > > > 1006. "If there are any failures in removing remote log
> segments
> > > then
> > > > > > those
> > > > > > > are stored in a specific topic (default as
> > > > > > __remote_segments_to_be_deleted)
> > > > > > > and user can consume the events(which contain
> > > remote-log-segment-id)
> > > > from
> > > > > > > that topic and clean them up from remote storage.  " Not sure
> if
> > > it's
> > > > > > worth
> > > > > > > the complexity of adding another topic. Could we just retry?
> > > > > > >
> > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the existing
> > > > > > > fetchPurgatory?
> > > > > > >
> > > > > > > 1008. Configurations:
> > > > > > > 1008.1 remote.log.retention.ms, remote.log.retention.minutes,
> > > > > > > remote.log.retention.hours: It seems that we just need the ms
> > one.
> > > > Also,
> > > > > > > are we changing the meaning of existing config
> log.retention.ms
> > to
> > > > mean
> > > > > > > the
> > > > > > > local retention? For backward compatibility, it's better to not
> > > > change
> > > > > > the
> > > > > > > meaning of existing configurations.
> > > > > > > 1008.2 Should remote.log.storage.enable be at the topic level?
> > > > > > >
> > > > > > > 1009. It would be useful to list all limitations in a separate
> > > > section:
> > > > > > > compacted topic, JBOD, etc. Also, is changing a topic from
> delete
> > > to
> > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > >
> > > > > > > 1010. Thanks for performance numbers. Are those with RocksDB as
> > the
> > > > > > cache?
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <harsha.ch@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > > > Hi Colin,
> > > > > > > >                Thats not what we said in the previous email.
> > RLMM
> > > > is
> > > > > > > > pluggable storage and by running numbers even 1PB data you do
> > not
> > > > need
> > > > > > > more
> > > > > > > > than 10GB local storage.
> > > > > > > > If in future this becomes a blocker for any users we can
> > revisit
> > > > but
> > > > > > this
> > > > > > > > does not warrant another implementation at this point to push
> > the
> > > > data
> > > > > > to
> > > > > > > > remote storage.
> > > > > > > > We can ofcourse implement another RLMM that is optional for
> > users
> > > > to
> > > > > > > > configure to push to remote. But that doesn't need to be
> > > addressed
> > > > in
> > > > > > > this
> > > > > > > > KIP.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Harsha
> > > > > > > >
> > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe <
> > cmccabe@apache.org
> > > >
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Ying,
> > > > > > > > >
> > > > > > > > > Thanks for the response.
> > > > > > > > >
> > > > > > > > > It sounds like you agree that storing the metadata in the
> > > remote
> > > > > > > storage
> > > > > > > > > would be a better design overall.  Given that that's true,
> is
> > > > there
> > > > > > any
> > > > > > > > > reason to include the worse implementation based on
> RocksDB?
> > > > > > > > >
> > > > > > > > > Choosing a long-term metadata store is not something that
> we
> > > > should
> > > > > > do
> > > > > > > > > lightly.  It can take users years to migrate from metadata
> > > store
> > > > to
> > > > > > the
> > > > > > > > > other.  I also don't think it's realistic or desirable for
> > > users
> > > > to
> > > > > > > write
> > > > > > > > > their own metadata stores.  Even assuming that they could
> do
> > a
> > > > good
> > > > > > job
> > > > > > > > at
> > > > > > > > > this, it would create huge fragmentation in the Kafka
> > > ecosystem.
> > > > > > > > >
> > > > > > > > > best,
> > > > > > > > > Colin
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng wrote:
> > > > > > > > > > Hi Jun,
> > > > > > > > > > Hi Colin,
> > > > > > > > > >
> > > > > > > > > > Satish and I are still discussing some details about how
> to
> > > > handle
> > > > > > > > > > transactions / producer ids. Satish is going to make some
> > > minor
> > > > > > > changes
> > > > > > > > > to
> > > > > > > > > > RLMM API and other parts. Other than that, we have
> finished
> > > > > > updating
> > > > > > > > the
> > > > > > > > > KIP
> > > > > > > > > >
> > > > > > > > > > I agree with Colin that the current design of using
> rocksDB
> > > is
> > > > not
> > > > > > > > > > optimal. But this design is simple and should work for
> > almost
> > > > all
> > > > > > the
> > > > > > > > > > existing Kafka users. RLMM is a plugin. Users can replace
> > > > rocksDB
> > > > > > > with
> > > > > > > > > > their own RLMM implementation, if needed. So, I think we
> > can
> > > > keep
> > > > > > > > rocksDB
> > > > > > > > > > for now. What do you think?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Ying
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> jun@confluent.io>
> > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi, Ying,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the update. It's good to see the progress on
> > > this.
> > > > > > > Please
> > > > > > > > > let us
> > > > > > > > > > > know when you are done updating the KIP wiki.
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng
> > > > > > <yingz@uber.com.invalid
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > >
> > > > > > > > > > > > Satish and I have added more design details in the
> KIP,
> > > > > > including
> > > > > > > > > how to
> > > > > > > > > > > > keep consistency between replicas (especially when
> > there
> > > is
> > > > > > > > > leadership
> > > > > > > > > > > > changes / log truncations) and new metrics. We also
> > made
> > > > some
> > > > > > > other
> > > > > > > > > minor
> > > > > > > > > > > > changes in the doc. We will finish the KIP changes in
> > the
> > > > next
> > > > > > > > > couple of
> > > > > > > > > > > > days. We will let you know when we are done. Most of
> > the
> > > > > > changes
> > > > > > > > are
> > > > > > > > > > > > already updated to the wiki KIP. You can take a look.
> > But
> > > > it's
> > > > > > > not
> > > > > > > > > the
> > > > > > > > > > > > final version yet.
> > > > > > > > > > > >
> > > > > > > > > > > > As for the implementation, the code is mostly done
> and
> > we
> > > > > > already
> > > > > > > > had
> > > > > > > > > > > some
> > > > > > > > > > > > feature tests / system tests. I have added the
> > > performance
> > > > test
> > > > > > > > > results
> > > > > > > > > > > in
> > > > > > > > > > > > the KIP. However the recent design changes (e.g.
> leader
> > > > epoch
> > > > > > > info
> > > > > > > > > > > > management / log truncation / some of the new
> metrics)
> > > > have not
> > > > > > > > been
> > > > > > > > > > > > implemented yet. It will take about 2 weeks for us to
> > > > implement
> > > > > > > > > after you
> > > > > > > > > > > > review and agree with those design changes.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun Rao <
> > jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Any new updates on the KIP? This feature is one of
> > the
> > > > most
> > > > > > > > > important
> > > > > > > > > > > and
> > > > > > > > > > > > > most requested features in Apache Kafka right now.
> It
> > > > would
> > > > > > be
> > > > > > > > > helpful
> > > > > > > > > > > if
> > > > > > > > > > > > > we can make sustained progress on this. Could you
> > share
> > > > how
> > > > > > far
> > > > > > > > > along
> > > > > > > > > > > is
> > > > > > > > > > > > > the design/implementation right now? Is there
> > anything
> > > > that
> > > > > > > other
> > > > > > > > > > > people
> > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > >
> > > > > > > > > > > > > As for "transactional support" and "follower
> > > > > > > > > requests/replication", no
> > > > > > > > > > > > > further comments from me as long as the producer
> > state
> > > > and
> > > > > > > leader
> > > > > > > > > epoch
> > > > > > > > > > > > can
> > > > > > > > > > > > > be restored properly from the object store when
> > needed.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM Satish Duggana <
> > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > We did not want to add many implementation
> details
> > in
> > > > the
> > > > > > > KIP.
> > > > > > > > > But we
> > > > > > > > > > > > > > decided to add them in the KIP as appendix or
> > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > follower fetch protocol) to describe the flow
> with
> > > the
> > > > main
> > > > > > > > > cases.
> > > > > > > > > > > > > > That will answer most of the queries. I will
> update
> > > on
> > > > this
> > > > > > > > mail
> > > > > > > > > > > > > > thread when the respective sections are updated.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM Alexandre Dupriez
> > > > > > > > > > > > > > <al...@gmail.com> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > A couple of questions specific to the section
> > > > "Follower
> > > > > > > > > > > > > > > Requests/Replication", pages 16:17 in the
> design
> > > > document
> > > > > > > > [1].
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900. It is mentioned that followers fetch
> > auxiliary
> > > > > > states
> > > > > > > > > from the
> > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900.a Does the consistency model of the
> external
> > > > storage
> > > > > > > > > impacts
> > > > > > > > > > > > reads
> > > > > > > > > > > > > > > of leader epochs and other auxiliary data?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900.b What are the benefits of using a
> mechanism
> > to
> > > > store
> > > > > > > and
> > > > > > > > > > > access
> > > > > > > > > > > > > > > the leader epochs which is different from other
> > > > metadata
> > > > > > > > > associated
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > tiered segments? What are the benefits of
> > > retrieving
> > > > this
> > > > > > > > > > > information
> > > > > > > > > > > > > > > on-demand from the follower rather than relying
> > on
> > > > > > > > propagation
> > > > > > > > > via
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > topic __remote_log_metadata? What are the
> > > advantages
> > > > over
> > > > > > > > > using a
> > > > > > > > > > > > > > > dedicated control structure (e.g. a new record
> > > type)
> > > > > > > > > propagated via
> > > > > > > > > > > > > > > this topic? Since in the document, different
> > > control
> > > > > > paths
> > > > > > > > are
> > > > > > > > > > > > > > > operating in the system, how are the metadata
> > > stored
> > > > in
> > > > > > > > > > > > > > > __remote_log_metadata [which also include the
> > epoch
> > > > of
> > > > > > the
> > > > > > > > > leader
> > > > > > > > > > > > > > > which offloaded a segment] and the remote
> > auxiliary
> > > > > > states,
> > > > > > > > > kept in
> > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900.c A follower can encounter an
> > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > Is
> > > > > > > > > > > > > > > this in response to a Fetch or
> > OffsetForLeaderEpoch
> > > > > > > request?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900.d What happens if, after a follower
> > encountered
> > > > an
> > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE response, its
> > > > attempts to
> > > > > > > > > retrieve
> > > > > > > > > > > > > > > leader epochs fail (for instance, because the
> > > remote
> > > > > > > storage
> > > > > > > > is
> > > > > > > > > > > > > > > temporarily unavailable)? Does the follower
> > > > fallbacks to
> > > > > > a
> > > > > > > > mode
> > > > > > > > > > > where
> > > > > > > > > > > > > > > it ignores tiered segments, and applies
> > truncation
> > > > using
> > > > > > > only
> > > > > > > > > > > locally
> > > > > > > > > > > > > > > available information? What happens when access
> > to
> > > > the
> > > > > > > remote
> > > > > > > > > > > storage
> > > > > > > > > > > > > > > is restored? How is the replica lineage
> inferred
> > by
> > > > the
> > > > > > > > remote
> > > > > > > > > > > leader
> > > > > > > > > > > > > > > epochs reconciled with the follower's replica
> > > > lineage,
> > > > > > > which
> > > > > > > > > has
> > > > > > > > > > > > > > > evolved? Does the follower remember fetching
> > > > auxiliary
> > > > > > > states
> > > > > > > > > > > failed
> > > > > > > > > > > > > > > in the past and attempt reconciliation? Is
> there
> > a
> > > > plan
> > > > > > to
> > > > > > > > > offer
> > > > > > > > > > > > > > > different strategies in this scenario,
> > configurable
> > > > via
> > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900.e Is the leader epoch cache offloaded with
> > > every
> > > > > > > segment?
> > > > > > > > > Or
> > > > > > > > > > > when
> > > > > > > > > > > > > > > a new checkpoint is detected? If that
> information
> > > is
> > > > not
> > > > > > > > always
> > > > > > > > > > > > > > > offloaded to avoid duplicating data, how does
> the
> > > > remote
> > > > > > > > > storage
> > > > > > > > > > > > > > > satisfy the request to retrieve it?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900.f Since the leader epoch cache covers the
> > > entire
> > > > > > > replica
> > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > what happens if, after a leader epoch cache
> file
> > is
> > > > > > > offloaded
> > > > > > > > > with
> > > > > > > > > > > a
> > > > > > > > > > > > > > > given segment, the local epoch cache is
> truncated
> > > > [not
> > > > > > > > > necessarily
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > a range of offset included in tiered segments]?
> > How
> > > > are
> > > > > > > > remote
> > > > > > > > > and
> > > > > > > > > > > > > > > local leader epoch caches kept consistent?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900.g Consumer can also use leader epochs (e.g.
> > to
> > > > enable
> > > > > > > > > fencing
> > > > > > > > > > > to
> > > > > > > > > > > > > > > protect against stale leaders). What
> differences
> > > > would
> > > > > > > there
> > > > > > > > be
> > > > > > > > > > > > > > > between consumer and follower fetches?
> > Especially,
> > > > would
> > > > > > > > > consumers
> > > > > > > > > > > > > > > also fetch leader epoch information from the
> > remote
> > > > > > > storage?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 900.h Assume a newly elected leader of a
> > > > topic-partition
> > > > > > > > > detects
> > > > > > > > > > > more
> > > > > > > > > > > > > > > recent segments are available in the external
> > > > storage,
> > > > > > with
> > > > > > > > > epochs
> > > > > > > > > > > >
> > > > > > > > > > > > > > > its local epoch. Does it ignore these segments
> > and
> > > > their
> > > > > > > > > associated
> > > > > > > > > > > > > > > epoch-to-offset vectors? Or try to reconstruct
> > its
> > > > local
> > > > > > > > > replica
> > > > > > > > > > > > > > > lineage based on the data remotely available?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55, Satish Duggana <
> > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > Please let us know if you have any comments
> on
> > > > > > > > "transactional
> > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > and "follower requests/replication" mentioned
> > in
> > > > the
> > > > > > > wiki.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25 PM Satish
> Duggana <
> > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks Jun for your comments.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >100. It would be useful to provide more
> > > details
> > > > on
> > > > > > how
> > > > > > > > > those
> > > > > > > > > > > > apis
> > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to really
> > > assess
> > > > > > > whether
> > > > > > > > > the
> > > > > > > > > > > new
> > > > > > > > > > > > > > apis are sufficient/redundant. A few examples
> > below.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > We will update the wiki and let you know.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >100.1 deleteRecords seems to only advance
> > the
> > > > > > > > > logStartOffset
> > > > > > > > > > > in
> > > > > > > > > > > > > > Log. How does that trigger the deletion of remote
> > log
> > > > > > > segments?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > RLMTask for leader partition periodically
> > > checks
> > > > > > > whether
> > > > > > > > > there
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > remote log segments earlier to
> logStartOffset
> > > > and the
> > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > remote log segment metadata and data are
> > > deleted
> > > > by
> > > > > > > using
> > > > > > > > > RLMM
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >100.2 stopReplica with deletion is used
> in 2
> > > > cases
> > > > > > (a)
> > > > > > > > > replica
> > > > > > > > > > > > > > reassignment; (b) topic deletion. We only want to
> > > > delete
> > > > > > the
> > > > > > > > > tiered
> > > > > > > > > > > > > > metadata in the second case. Also, in the second
> > > case,
> > > > who
> > > > > > > > > initiates
> > > > > > > > > > > > the
> > > > > > > > > > > > > > deletion of the remote segment since the leader
> may
> > > not
> > > > > > > exist?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Right, it is deleted only incase of topic
> > > > deletion
> > > > > > > only.
> > > > > > > > We
> > > > > > > > > > > will
> > > > > > > > > > > > > > cover
> > > > > > > > > > > > > > > > > the details in the KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >100.3 "LogStartOffset of a topic can be
> > either
> > > > in
> > > > > > > local
> > > > > > > > > or in
> > > > > > > > > > > > > > remote storage." If LogStartOffset exists in both
> > > > places,
> > > > > > > which
> > > > > > > > > one
> > > > > > > > > > > is
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > source of truth?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I meant the logStartOffset can point to
> > either
> > > of
> > > > > > local
> > > > > > > > > segment
> > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > remote segment but it is initialised and
> > > > maintained
> > > > > > in
> > > > > > > > the
> > > > > > > > > Log
> > > > > > > > > > > > > class
> > > > > > > > > > > > > > > > > like now.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >100.4 List<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > topicPartition,
> > > > long
> > > > > > > > > minOffset):
> > > > > > > > > > > > How
> > > > > > > > > > > > > > is minOffset supposed to be used?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Returns list of remote segments, sorted by
> > > > baseOffset
> > > > > > > in
> > > > > > > > > > > > ascending
> > > > > > > > > > > > > > > > > order that have baseOffset >= the given min
> > > > Offset.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >100.5 When copying a segment to remote
> > > storage,
> > > > it
> > > > > > > seems
> > > > > > > > > we
> > > > > > > > > > > are
> > > > > > > > > > > > > > calling the same RLMM.putRemoteLogSegmentData()
> > twice
> > > > > > before
> > > > > > > > and
> > > > > > > > > > > after
> > > > > > > > > > > > > > copyLogSegment(). Could you explain why?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > This is more about prepare/commit/rollback
> as
> > > you
> > > > > > > > > suggested.
> > > > > > > > > > > We
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > update the wiki with the new APIs.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >100.6 LogSegmentData includes
> > > leaderEpochCache,
> > > > but
> > > > > > > > there
> > > > > > > > > is
> > > > > > > > > > > no
> > > > > > > > > > > > > api
> > > > > > > > > > > > > > in RemoteStorageManager to retrieve it.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Nice catch, copy/paste issue. There is an
> API
> > > to
> > > > > > > retrieve
> > > > > > > > > it.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >101. If the __remote_log_metadata is for
> > > > production
> > > > > > > > usage,
> > > > > > > > > > > could
> > > > > > > > > > > > > > you provide more details? For example, what is
> the
> > > > schema
> > > > > > of
> > > > > > > > the
> > > > > > > > > data
> > > > > > > > > > > > > (both
> > > > > > > > > > > > > > key and value)? How is the topic
> maintained,delete
> > or
> > > > > > > compact?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > It is with delete config and it’s retention
> > > > period is
> > > > > > > > > suggested
> > > > > > > > > > > > to
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > more than the remote retention period.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >110. Is the cache implementation in
> > > > > > > > > RemoteLogMetadataManager
> > > > > > > > > > > > meant
> > > > > > > > > > > > > > for production usage? If so, could you provide
> more
> > > > details
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > schema
> > > > > > > > > > > > > > and how/where the data is stored?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The proposal is to have a cache (with
> default
> > > > > > > > > implementation
> > > > > > > > > > > > backed
> > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > rocksdb) but it will be added in later
> > > versions.
> > > > We
> > > > > > > will
> > > > > > > > > add
> > > > > > > > > > > this
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > future work items.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >111. "Committed offsets can be stored in a
> > > local
> > > > > > > file".
> > > > > > > > > Could
> > > > > > > > > > > > you
> > > > > > > > > > > > > > describe the format of the file and where it's
> > > stored?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > We will cover this in the KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >112. Truncation of remote segments under
> > > unclean
> > > > > > > leader
> > > > > > > > > > > > election:
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > am not sure who figures out the truncated remote
> > > > segments
> > > > > > and
> > > > > > > > how
> > > > > > > > > > > that
> > > > > > > > > > > > > > information is propagated to all replicas?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > We will add this in detail in the KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >113. "If there are any failures in
> removing
> > > > remote
> > > > > > log
> > > > > > > > > > > segments
> > > > > > > > > > > > > > then those are stored in a specific topic
> (default
> > as
> > > > > > > > > > > > > > __remote_segments_to_be_deleted)". Is it
> necessary
> > to
> > > > add
> > > > > > yet
> > > > > > > > > another
> > > > > > > > > > > > > > internal topic? Could we just keep retrying?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > This is not really an internal topic, it
> will
> > > be
> > > > > > > exposed
> > > > > > > > > as a
> > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > configurable topic. After a few retries, we
> > > want
> > > > user
> > > > > > > to
> > > > > > > > > know
> > > > > > > > > > > > about
> > > > > > > > > > > > > > > > > the failure so that they can take an action
> > > > later by
> > > > > > > > > consuming
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > this topic. We want to keep this simple
> > instead
> > > > of
> > > > > > > > retrying
> > > > > > > > > > > > > > > > > continuously and maintaining the deletion
> > state
> > > > etc.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >114. "We may not need to copy
> > > > producer-id-snapshot
> > > > > > as
> > > > > > > we
> > > > > > > > > are
> > > > > > > > > > > > > > copying only segments earlier to
> > last-stable-offset."
> > > > Hmm,
> > > > > > > not
> > > > > > > > > sure
> > > > > > > > > > > > about
> > > > > > > > > > > > > > that. The producer snapshot includes things like
> > the
> > > > last
> > > > > > > > > timestamp
> > > > > > > > > > > of
> > > > > > > > > > > > > each
> > > > > > > > > > > > > > open producer id and can affect when those
> producer
> > > > ids are
> > > > > > > > > expired.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Sure, this will be added as part of the
> > > > > > LogSegmentData.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Fri, May 29, 2020 at 6:39 AM Jun Rao <
> > > > > > > > jun@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Made another pass on the wiki. A few more
> > > > comments
> > > > > > > > below.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 100. It would be useful to provide more
> > > > details on
> > > > > > > how
> > > > > > > > > those
> > > > > > > > > > > > apis
> > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to really
> > > assess
> > > > > > > whether
> > > > > > > > > the
> > > > > > > > > > > new
> > > > > > > > > > > > > > apis are sufficient/redundant. A few examples
> > below.
> > > > > > > > > > > > > > > > > > 100.1 deleteRecords seems to only advance
> > the
> > > > > > > > > logStartOffset
> > > > > > > > > > > in
> > > > > > > > > > > > > > Log. How does that trigger the deletion of remote
> > log
> > > > > > > segments?
> > > > > > > > > > > > > > > > > > 100.2 stopReplica with deletion is used
> in
> > 2
> > > > cases
> > > > > > > (a)
> > > > > > > > > > > replica
> > > > > > > > > > > > > > reassignment; (b) topic deletion. We only want to
> > > > delete
> > > > > > the
> > > > > > > > > tiered
> > > > > > > > > > > > > > metadata in the second case. Also, in the second
> > > case,
> > > > who
> > > > > > > > > initiates
> > > > > > > > > > > > the
> > > > > > > > > > > > > > deletion of the remote segment since the leader
> may
> > > not
> > > > > > > exist?
> > > > > > > > > > > > > > > > > > 100.3 "LogStartOffset of a topic can be
> > > either
> > > > in
> > > > > > > local
> > > > > > > > > or in
> > > > > > > > > > > > > > remote storage." If LogStartOffset exists in both
> > > > places,
> > > > > > > which
> > > > > > > > > one
> > > > > > > > > > > is
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > source of truth?
> > > > > > > > > > > > > > > > > > 100.4 List<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > topicPartition,
> > > > long
> > > > > > > > > minOffset):
> > > > > > > > > > > > How
> > > > > > > > > > > > > > is minOffset supposed to be used?
> > > > > > > > > > > > > > > > > > 100.5 When copying a segment to remote
> > > > storage, it
> > > > > > > > seems
> > > > > > > > > we
> > > > > > > > > > > are
> > > > > > > > > > > > > > calling the same RLMM.putRemoteLogSegmentData()
> > twice
> > > > > > before
> > > > > > > > and
> > > > > > > > > > > after
> > > > > > > > > > > > > > copyLogSegment(). Could you explain why?
> > > > > > > > > > > > > > > > > > 100.6 LogSegmentData includes
> > > > leaderEpochCache, but
> > > > > > > > > there is
> > > > > > > > > > > no
> > > > > > > > > > > > > > api in RemoteStorageManager to retrieve it.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 101. If the __remote_log_metadata is for
> > > > production
> > > > > > > > > usage,
> > > > > > > > > > > > could
> > > > > > > > > > > > > > you provide more details? For example, what is
> the
> > > > schema
> > > > > > of
> > > > > > > > the
> > > > > > > > > data
> > > > > > > > > > > > > (both
> > > > > > > > > > > > > > key and value)? How is the topic
> maintained,delete
> > or
> > > > > > > compact?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 110. Is the cache implementation in
> > > > > > > > > RemoteLogMetadataManager
> > > > > > > > > > > > > meant
> > > > > > > > > > > > > > for production usage? If so, could you provide
> more
> > > > details
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > schema
> > > > > > > > > > > > > > and how/where the data is stored?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 111. "Committed offsets can be stored in
> a
> > > > local
> > > > > > > file".
> > > > > > > > > Could
> > > > > > > > > > > > you
> > > > > > > > > > > > > > describe the format of the file and where it's
> > > stored?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 112. Truncation of remote segments under
> > > > unclean
> > > > > > > leader
> > > > > > > > > > > > election:
> > > > > > > > > > > > > > I am not sure who figures out the truncated
> remote
> > > > segments
> > > > > > > and
> > > > > > > > > how
> > > > > > > > > > > > that
> > > > > > > > > > > > > > information is propagated to all replicas?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 113. "If there are any failures in
> removing
> > > > remote
> > > > > > > log
> > > > > > > > > > > segments
> > > > > > > > > > > > > > then those are stored in a specific topic
> (default
> > as
> > > > > > > > > > > > > > __remote_segments_to_be_deleted)". Is it
> necessary
> > to
> > > > add
> > > > > > yet
> > > > > > > > > another
> > > > > > > > > > > > > > internal topic? Could we just keep retrying?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 114. "We may not need to copy
> > > > producer-id-snapshot
> > > > > > as
> > > > > > > > we
> > > > > > > > > are
> > > > > > > > > > > > > > copying only segments earlier to
> > last-stable-offset."
> > > > Hmm,
> > > > > > > not
> > > > > > > > > sure
> > > > > > > > > > > > about
> > > > > > > > > > > > > > that. The producer snapshot includes things like
> > the
> > > > last
> > > > > > > > > timestamp
> > > > > > > > > > > of
> > > > > > > > > > > > > each
> > > > > > > > > > > > > > open producer id and can affect when those
> producer
> > > > ids are
> > > > > > > > > expired.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Thu, May 28, 2020 at 5:38 AM Satish
> > > Duggana
> > > > <
> > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> Hi Jun,
> > > > > > > > > > > > > > > > > >> Gentle reminder. Please go through the
> > > updated
> > > > > > wiki
> > > > > > > > and
> > > > > > > > > let
> > > > > > > > > > > us
> > > > > > > > > > > > > > know your comments.
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> Thanks,
> > > > > > > > > > > > > > > > > >> Satish.
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> On Tue, May 19, 2020 at 3:50 PM Satish
> > > > Duggana <
> > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > >>> Hi Jun,
> > > > > > > > > > > > > > > > > >>> Please go through the wiki which has
> the
> > > > latest
> > > > > > > > > updates.
> > > > > > > > > > > > Google
> > > > > > > > > > > > > > doc is updated frequently to be in sync with
> wiki.
> > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > >>> Thanks,
> > > > > > > > > > > > > > > > > >>> Satish.
> > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > >>> On Tue, May 19, 2020 at 12:30 AM Jun
> Rao
> > <
> > > > > > > > > jun@confluent.io
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > >>>> Hi, Satish,
> > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > >>>> Thanks for the update. Just to
> clarify.
> > > > Which
> > > > > > doc
> > > > > > > > has
> > > > > > > > > the
> > > > > > > > > > > > > > latest updates, the wiki or the google doc?
> > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > >>>> Jun
> > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > >>>> On Thu, May 14, 2020 at 10:38 AM
> Satish
> > > > Duggana
> > > > > > <
> > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> Hi Jun,
> > > > > > > > > > > > > > > > > >>>>> Thanks for your comments.  We updated
> > the
> > > > KIP
> > > > > > > with
> > > > > > > > > more
> > > > > > > > > > > > > > details.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> >100. For each of the operations
> > related
> > > to
> > > > > > > > tiering,
> > > > > > > > > it
> > > > > > > > > > > > would
> > > > > > > > > > > > > > be useful to provide a description on how it
> works
> > > > with the
> > > > > > > new
> > > > > > > > > API.
> > > > > > > > > > > > > These
> > > > > > > > > > > > > > include things like consumer fetch, replica
> fetch,
> > > > > > > > > > > offsetForTimestamp,
> > > > > > > > > > > > > > retention (remote and local) by size, time and
> > > > > > > logStartOffset,
> > > > > > > > > topic
> > > > > > > > > > > > > > deletion, etc. This will tell us if the proposed
> > APIs
> > > > are
> > > > > > > > > sufficient.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> We addressed most of these APIs in
> the
> > > > KIP. We
> > > > > > > can
> > > > > > > > > add
> > > > > > > > > > > more
> > > > > > > > > > > > > > details if needed.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> >101. For the default implementation
> > > based
> > > > on
> > > > > > > > > internal
> > > > > > > > > > > > topic,
> > > > > > > > > > > > > > is it meant as a proof of concept or for
> production
> > > > usage?
> > > > > > I
> > > > > > > > > assume
> > > > > > > > > > > > that
> > > > > > > > > > > > > > it's the former. However, if it's the latter,
> then
> > > the
> > > > KIP
> > > > > > > > needs
> > > > > > > > > to
> > > > > > > > > > > > > > describe the design in more detail.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> It is production usage as was
> mentioned
> > > in
> > > > an
> > > > > > > > earlier
> > > > > > > > > > > mail.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > plan to update this section in the next few days.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> >102. When tiering a segment, the
> > segment
> > > > is
> > > > > > > first
> > > > > > > > > > > written
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the object store and then its metadata is written
> > to
> > > > RLMM
> > > > > > > using
> > > > > > > > > the
> > > > > > > > > > > api
> > > > > > > > > > > > > > "void putRemoteLogSegmentData()". One potential
> > issue
> > > > with
> > > > > > > this
> > > > > > > > > > > > approach
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > that if the system fails after the first
> operation,
> > > it
> > > > > > > leaves a
> > > > > > > > > > > garbage
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > the object store that's never reclaimed. One way
> to
> > > > improve
> > > > > > > > this
> > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > two separate APIs, sth like
> > > > > > preparePutRemoteLogSegmentData()
> > > > > > > > and
> > > > > > > > > > > > > > commitPutRemoteLogSegmentData().
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> That is a good point. We currently
> > have a
> > > > > > > different
> > > > > > > > > way
> > > > > > > > > > > > using
> > > > > > > > > > > > > > markers in the segment but your suggestion is
> much
> > > > better.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> >103. It seems that the transactional
> > > > support
> > > > > > and
> > > > > > > > the
> > > > > > > > > > > > ability
> > > > > > > > > > > > > > to read from follower are missing.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> KIP is updated with transactional
> > > support,
> > > > > > > follower
> > > > > > > > > fetch
> > > > > > > > > > > > > > semantics, and reading from a follower.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> >104. It would be useful to provide a
> > > > testing
> > > > > > > plan
> > > > > > > > > for
> > > > > > > > > > > this
> > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> We added a few tests by introducing
> > test
> > > > util
> > > > > > for
> > > > > > > > > tiered
> > > > > > > > > > > > > > storage in the PR. We will provide the testing
> plan
> > > in
> > > > the
> > > > > > > next
> > > > > > > > > few
> > > > > > > > > > > > days.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> Thanks,
> > > > > > > > > > > > > > > > > >>>>> Satish.
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > >>>>> On Wed, Feb 26, 2020 at 9:43 PM
> Harsha
> > > > > > > > Chintalapani <
> > > > > > > > > > > > > > kafka@harsha.io> wrote:
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>> On Tue, Feb 25, 2020 at 12:46 PM,
> Jun
> > > Rao
> > > > <
> > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks for the updated doc. The new
> > API
> > > > seems
> > > > > > > to
> > > > > > > > > be an
> > > > > > > > > > > > > > improvement overall. A few more comments below.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 100. For each of the operations
> > related
> > > > to
> > > > > > > > > tiering, it
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > be useful to provide a description on how it
> works
> > > > with the
> > > > > > > new
> > > > > > > > > API.
> > > > > > > > > > > > > These
> > > > > > > > > > > > > > include things like consumer fetch, replica
> fetch,
> > > > > > > > > > > offsetForTimestamp,
> > > > > > > > > > > > > > retention
> > > > > > > > > > > > > > > > > >>>>>>> (remote and local) by size, time
> and
> > > > > > > > > logStartOffset,
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > deletion, etc. This will tell us if the proposed
> > APIs
> > > > are
> > > > > > > > > sufficient.
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>> Thanks for the feedback Jun. We will
> > add
> > > > more
> > > > > > > > > details
> > > > > > > > > > > > around
> > > > > > > > > > > > > > this.
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 101. For the default implementation
> > > > based on
> > > > > > > > > internal
> > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > is it meant as a proof of concept or for
> production
> > > > usage?
> > > > > > I
> > > > > > > > > assume
> > > > > > > > > > > > that
> > > > > > > > > > > > > > it's the former. However, if it's the latter,
> then
> > > the
> > > > KIP
> > > > > > > > needs
> > > > > > > > > to
> > > > > > > > > > > > > > describe the design in more detail.
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>> Yes it meant to be for production
> use.
> > > > > > Ideally
> > > > > > > it
> > > > > > > > > would
> > > > > > > > > > > > be
> > > > > > > > > > > > > > good to merge this in as the default
> implementation
> > > for
> > > > > > > > metadata
> > > > > > > > > > > > service.
> > > > > > > > > > > > > > We can add more details around design and
> testing.
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 102. When tiering a segment, the
> > > segment
> > > > is
> > > > > > > first
> > > > > > > > > > > written
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > the object store and then its metadata is written
> > to
> > > > RLMM
> > > > > > > using
> > > > > > > > > the
> > > > > > > > > > > api
> > > > > > > > > > > > > > "void putRemoteLogSegmentData()".
> > > > > > > > > > > > > > > > > >>>>>>> One potential issue with this
> > approach
> > > is
> > > > > > that
> > > > > > > if
> > > > > > > > > the
> > > > > > > > > > > > > system
> > > > > > > > > > > > > > fails after the first operation, it leaves a
> > garbage
> > > > in the
> > > > > > > > > object
> > > > > > > > > > > > store
> > > > > > > > > > > > > > that's never reclaimed. One way to improve this
> is
> > to
> > > > have
> > > > > > > two
> > > > > > > > > > > separate
> > > > > > > > > > > > > > APIs, sth like preparePutRemoteLogSegmentData()
> and
> > > > > > > > > > > > > > commitPutRemoteLogSegmentData().
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 103. It seems that the
> transactional
> > > > support
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > > > ability
> > > > > > > > > > > > > > to read from follower are missing.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 104. It would be useful to provide
> a
> > > > testing
> > > > > > > plan
> > > > > > > > > for
> > > > > > > > > > > > this
> > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>> We are working on adding more
> details
> > > > around
> > > > > > > > > > > transactional
> > > > > > > > > > > > > > support and coming up with test plan.
> > > > > > > > > > > > > > > > > >>>>>> Add system tests and integration
> > tests.
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Mon, Feb 24, 2020 at 8:10 AM
> > Satish
> > > > > > Duggana
> > > > > > > <
> > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > >>>>>>> Please look at the earlier reply
> and
> > > let
> > > > us
> > > > > > > know
> > > > > > > > > your
> > > > > > > > > > > > > > comments.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > >>>>>>> Satish.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Wed, Feb 12, 2020 at 4:06 PM
> > Satish
> > > > > > Duggana
> > > > > > > <
> > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > >>>>>>> Thanks for your comments on the
> > > > separation of
> > > > > > > > > remote
> > > > > > > > > > > log
> > > > > > > > > > > > > > metadata storage and remote log storage.
> > > > > > > > > > > > > > > > > >>>>>>> We had a few discussions since
> early
> > > Jan
> > > > on
> > > > > > how
> > > > > > > > to
> > > > > > > > > > > > support
> > > > > > > > > > > > > > eventually consistent stores like S3 by
> uncoupling
> > > > remote
> > > > > > log
> > > > > > > > > segment
> > > > > > > > > > > > > > metadata and remote log storage. It is written
> with
> > > > details
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > doc
> > > > > > > > > > > > > > here(1). Below is the brief summary of the
> > discussion
> > > > from
> > > > > > > that
> > > > > > > > > doc.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> The current approach consists of
> > > pulling
> > > > the
> > > > > > > > > remote log
> > > > > > > > > > > > > > segment metadata from remote log storage APIs. It
> > > > worked
> > > > > > fine
> > > > > > > > for
> > > > > > > > > > > > > storages
> > > > > > > > > > > > > > like HDFS. But one of the problems of relying on
> > the
> > > > remote
> > > > > > > > > storage
> > > > > > > > > > > to
> > > > > > > > > > > > > > maintain metadata is that tiered-storage needs to
> > be
> > > > > > strongly
> > > > > > > > > > > > consistent,
> > > > > > > > > > > > > > with an impact not only on the metadata(e.g. LIST
> > in
> > > > S3)
> > > > > > but
> > > > > > > > > also on
> > > > > > > > > > > > the
> > > > > > > > > > > > > > segment data(e.g. GET after a DELETE in S3). The
> > cost
> > > > of
> > > > > > > > > maintaining
> > > > > > > > > > > > > > metadata in remote storage needs to be factored
> in.
> > > > This is
> > > > > > > > true
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > case of S3, LIST APIs incur huge costs as you
> > raised
> > > > > > earlier.
> > > > > > > > > > > > > > > > > >>>>>>> So, it is good to separate the
> remote
> > > > storage
> > > > > > > > from
> > > > > > > > > the
> > > > > > > > > > > > > > remote log metadata store. We refactored the
> > existing
> > > > > > > > > > > > > RemoteStorageManager
> > > > > > > > > > > > > > and introduced RemoteLogMetadataManager. Remote
> log
> > > > > > metadata
> > > > > > > > > store
> > > > > > > > > > > > should
> > > > > > > > > > > > > > give strong consistency semantics but remote log
> > > > storage
> > > > > > can
> > > > > > > be
> > > > > > > > > > > > > eventually
> > > > > > > > > > > > > > consistent.
> > > > > > > > > > > > > > > > > >>>>>>> We can have a default
> implementation
> > > for
> > > > > > > > > > > > > > RemoteLogMetadataManager which uses an internal
> > > > topic(as
> > > > > > > > > mentioned in
> > > > > > > > > > > > one
> > > > > > > > > > > > > > of our earlier emails) as storage. But users can
> > > always
> > > > > > > plugin
> > > > > > > > > their
> > > > > > > > > > > > own
> > > > > > > > > > > > > > RemoteLogMetadataManager implementation based on
> > > their
> > > > > > > > > environment.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Please go through the updated KIP
> and
> > > > let us
> > > > > > > know
> > > > > > > > > your
> > > > > > > > > > > > > > comments. We have started refactoring for the
> > changes
> > > > > > > mentioned
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > and there may be a few more updates to the APIs.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> [1]
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/1qfkBCWL1e7ZWkHU7brxKDBebq4ie9yK20XJnKbgAlew/edit?ts=5e208ec7#
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Fri, Dec 27, 2019 at 5:43 PM
> Ivan
> > > > > > Yurchenko
> > > > > > > <
> > > > > > > > > > > > > > ivan0yurchenko@gmail.com>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi all,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Jun:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (a) Cost: S3 list object requests
> > cost
> > > > $0.005
> > > > > > > per
> > > > > > > > > 1000
> > > > > > > > > > > > > > requests. If
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> have 100,000 partitions and want to
> > > pull
> > > > the
> > > > > > > > > metadata
> > > > > > > > > > > for
> > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> partition
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> at
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the rate of 1/sec. It can cost
> > > $0.5/sec,
> > > > > > which
> > > > > > > is
> > > > > > > > > > > roughly
> > > > > > > > > > > > > > $40K per
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> day.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> I want to note here, that no
> > reasonably
> > > > > > durable
> > > > > > > > > storage
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > be cheap at 100k RPS. For example, DynamoDB might
> > > give
> > > > the
> > > > > > > same
> > > > > > > > > > > > ballpark
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> figures.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> If we want to keep the pull-based
> > > > approach,
> > > > > > we
> > > > > > > > can
> > > > > > > > > try
> > > > > > > > > > > to
> > > > > > > > > > > > > > reduce this
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> number
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> in several ways: doing listings
> less
> > > > > > frequently
> > > > > > > > (as
> > > > > > > > > > > > Satish
> > > > > > > > > > > > > > mentioned, with the current defaults it's ~3.33k
> > RPS
> > > > for
> > > > > > your
> > > > > > > > > > > example),
> > > > > > > > > > > > > > batching listing operations in some way
> (depending
> > on
> > > > the
> > > > > > > > > storage; it
> > > > > > > > > > > > > might
> > > > > > > > > > > > > > require the change of RSM's interface).
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> There are different ways for doing
> > push
> > > > based
> > > > > > > > > metadata
> > > > > > > > > > > > > > propagation.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Some
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> object stores may support that
> > already.
> > > > For
> > > > > > > > > example, S3
> > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> events
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> notification
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> This sounds interesting. However, I
> > > see a
> > > > > > > couple
> > > > > > > > of
> > > > > > > > > > > > issues
> > > > > > > > > > > > > > using it:
> > > > > > > > > > > > > > > > > >>>>>>> 1. As I understand the
> documentation,
> > > > > > > > notification
> > > > > > > > > > > > delivery
> > > > > > > > > > > > > > is not guaranteed
> > > > > > > > > > > > > > > > > >>>>>>> and it's recommended to
> periodically
> > do
> > > > LIST
> > > > > > to
> > > > > > > > > fill
> > > > > > > > > > > the
> > > > > > > > > > > > > > gaps. Which brings us back to the same LIST
> > > consistency
> > > > > > > > > guarantees
> > > > > > > > > > > > issue.
> > > > > > > > > > > > > > > > > >>>>>>> 2. The same goes for the broker
> > start:
> > > > to get
> > > > > > > the
> > > > > > > > > > > current
> > > > > > > > > > > > > > state, we
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> need
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to LIST.
> > > > > > > > > > > > > > > > > >>>>>>> 3. The dynamic set of multiple
> > > consumers
> > > > > > > (RSMs):
> > > > > > > > > AFAIK
> > > > > > > > > > > > SQS
> > > > > > > > > > > > > > and SNS
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> aren't
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> designed for such a case.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Alexandre:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> A.1 As commented on PR 7561, S3
> > > > consistency
> > > > > > > model
> > > > > > > > > > > [1][2]
> > > > > > > > > > > > > > implies RSM
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> cannot
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> relies solely on S3 APIs to
> guarantee
> > > the
> > > > > > > > expected
> > > > > > > > > > > strong
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> consistency. The
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> proposed implementation [3] would
> > need
> > > > to be
> > > > > > > > > updated to
> > > > > > > > > > > > > take
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> into
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> account. Let’s talk more about
> this.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thank you for the feedback. I
> clearly
> > > > see the
> > > > > > > > need
> > > > > > > > > for
> > > > > > > > > > > > > > changing the S3 implementation
> > > > > > > > > > > > > > > > > >>>>>>> to provide stronger consistency
> > > > guarantees.
> > > > > > As
> > > > > > > it
> > > > > > > > > see
> > > > > > > > > > > > from
> > > > > > > > > > > > > > this thread, there are
> > > > > > > > > > > > > > > > > >>>>>>> several possible approaches to
> this.
> > > > Let's
> > > > > > > > discuss
> > > > > > > > > > > > > > RemoteLogManager's contract and
> > > > > > > > > > > > > > > > > >>>>>>> behavior (like pull vs push model)
> > > > further
> > > > > > > before
> > > > > > > > > > > picking
> > > > > > > > > > > > > > one (or
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> several -
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> ?) of them.
> > > > > > > > > > > > > > > > > >>>>>>> I'm going to do some evaluation of
> > > > DynamoDB
> > > > > > for
> > > > > > > > the
> > > > > > > > > > > > > > pull-based
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> approach,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> if it's possible to apply it
> paying a
> > > > > > > reasonable
> > > > > > > > > bill.
> > > > > > > > > > > > > Also,
> > > > > > > > > > > > > > of the push-based approach
> > > > > > > > > > > > > > > > > >>>>>>> with a Kafka topic as the medium.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> A.2.3 Atomicity – what does an
> > > > implementation
> > > > > > > of
> > > > > > > > > RSM
> > > > > > > > > > > need
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> respect to atomicity of the APIs
> > > > > > > copyLogSegment,
> > > > > > > > > > > > > > cleanupLogUntil and deleteTopicPartition? If a
> > > partial
> > > > > > > failure
> > > > > > > > > > > happens
> > > > > > > > > > > > in
> > > > > > > > > > > > > > any of those
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (e.g.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the S3 implementation, if one of
> the
> > > > multiple
> > > > > > > > > uploads
> > > > > > > > > > > > fails
> > > > > > > > > > > > > > [4]),
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> The S3 implementation is going to
> > > > change, but
> > > > > > > > it's
> > > > > > > > > > > worth
> > > > > > > > > > > > > > clarifying
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> anyway.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> The segment log file is being
> > uploaded
> > > > after
> > > > > > S3
> > > > > > > > has
> > > > > > > > > > > acked
> > > > > > > > > > > > > > uploading of all other files associated with the
> > > > segment
> > > > > > and
> > > > > > > > only
> > > > > > > > > > > after
> > > > > > > > > > > > > > this the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> whole
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> segment file set becomes visible
> > > > remotely for
> > > > > > > > > > > operations
> > > > > > > > > > > > > > like listRemoteSegments [1].
> > > > > > > > > > > > > > > > > >>>>>>> In case of upload failure, the
> files
> > > > that has
> > > > > > > > been
> > > > > > > > > > > > > > successfully
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> uploaded
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> stays
> > > > > > > > > > > > > > > > > >>>>>>> as invisible garbage that is
> > collected
> > > by
> > > > > > > > > > > cleanupLogUntil
> > > > > > > > > > > > > (or
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> overwritten
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> successfully later).
> > > > > > > > > > > > > > > > > >>>>>>> And the opposite happens during the
> > > > deletion:
> > > > > > > log
> > > > > > > > > files
> > > > > > > > > > > > are
> > > > > > > > > > > > > > deleted
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> first.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> This approach should generally work
> > > when
> > > > we
> > > > > > > solve
> > > > > > > > > > > > > > consistency issues by adding a strongly
> consistent
> > > > > > storage: a
> > > > > > > > > > > segment's
> > > > > > > > > > > > > > uploaded files
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> remain
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> invisible garbage until some
> metadata
> > > > about
> > > > > > > them
> > > > > > > > is
> > > > > > > > > > > > > written.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> A.3 Caching – storing locally the
> > > > segments
> > > > > > > > > retrieved
> > > > > > > > > > > from
> > > > > > > > > > > > > > the remote storage is excluded as it does not
> align
> > > > with
> > > > > > the
> > > > > > > > > original
> > > > > > > > > > > > > intent
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> and even
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> defeat some of its purposes (save
> > disk
> > > > space
> > > > > > > > etc.).
> > > > > > > > > > > That
> > > > > > > > > > > > > > said, could
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> there
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> be other types of use cases where
> the
> > > > pattern
> > > > > > > of
> > > > > > > > > access
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> remotely
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> stored segments would benefit from
> > > local
> > > > > > > caching
> > > > > > > > > (and
> > > > > > > > > > > > > > potentially read-ahead)? Consider the use case
> of a
> > > > large
> > > > > > > pool
> > > > > > > > of
> > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> start
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> a backfill at the same time for one
> > day
> > > > worth
> > > > > > > of
> > > > > > > > > data
> > > > > > > > > > > > from
> > > > > > > > > > > > > > one year
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> ago
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> stored remotely. Caching the
> segments
> > > > locally
> > > > > > > > would
> > > > > > > > > > > allow
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> uncouple the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> load on the remote storage from the
> > > load
> > > > on
> > > > > > the
> > > > > > > > > Kafka
> > > > > > > > > > > > > > cluster. Maybe
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> RLM could expose a configuration
> > > > parameter to
> > > > > > > > > switch
> > > > > > > > > > > that
> > > > > > > > > > > > > > feature
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> on/off?
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> I tend to agree here, caching
> remote
> > > > segments
> > > > > > > > > locally
> > > > > > > > > > > and
> > > > > > > > > > > > > > making this configurable sounds pretty practical
> to
> > > > me. We
> > > > > > > > should
> > > > > > > > > > > > > implement
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> this,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> maybe not in the first iteration.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Br,
> > > > > > > > > > > > > > > > > >>>>>>> Ivan
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> [1]
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://github.com/harshach/kafka/pull/18/files#diff-4d73d01c16caed6f2548fc3063550ef0R152
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Thu, 19 Dec 2019 at 19:49,
> > Alexandre
> > > > > > > Dupriez <
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> alexandre.dupriez@gmail.com>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thank you for the feedback. I am
> > trying
> > > > to
> > > > > > > > > understand
> > > > > > > > > > > > how a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> push-based
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> approach would work.
> > > > > > > > > > > > > > > > > >>>>>>> In order for the metadata to be
> > > > propagated
> > > > > > > (under
> > > > > > > > > the
> > > > > > > > > > > > > > assumption you stated), would you plan to add a
> new
> > > > API in
> > > > > > > > Kafka
> > > > > > > > > to
> > > > > > > > > > > > allow
> > > > > > > > > > > > > > the metadata store to send them directly to the
> > > > brokers?
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > >>>>>>> Alexandre
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Le mer. 18 déc. 2019 à 20:14, Jun
> > Rao <
> > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > a
> > > > > > > > > > > > > > écrit :
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Ivan,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> There are different ways for doing
> > push
> > > > based
> > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> propagation. Some
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> object stores may support that
> > already.
> > > > For
> > > > > > > > > example, S3
> > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> events
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> notification (
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > >
> > > >
> https://docs.aws.amazon.com/AmazonS3/latest/dev/NotificationHowTo.html
> > > > > > > > > > > > ).
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Otherwise one could use a separate
> > > > metadata
> > > > > > > store
> > > > > > > > > that
> > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> push-based
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> change propagation. Other people
> have
> > > > > > mentioned
> > > > > > > > > using a
> > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> topic. The
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> best approach may depend on the
> > object
> > > > store
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > > > > operational environment (e.g. whether an external
> > > > metadata
> > > > > > > > store
> > > > > > > > > is
> > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> available).
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> The above discussion is based on
> the
> > > > > > assumption
> > > > > > > > > that we
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> cache the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> object metadata locally in every
> > > broker.
> > > > I
> > > > > > > > > mentioned
> > > > > > > > > > > > > earlier
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> an
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> alternative is to just
> store/retrieve
> > > > those
> > > > > > > > > metadata in
> > > > > > > > > > > > an
> > > > > > > > > > > > > > external metadata store. That may simplify the
> > > > > > implementation
> > > > > > > > in
> > > > > > > > > some
> > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Thu, Dec 5, 2019 at 7:01 AM
> Satish
> > > > > > Duggana <
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > >>>>>>> Thanks for your reply.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Currently, `listRemoteSegments` is
> > > > called at
> > > > > > > the
> > > > > > > > > > > > configured
> > > > > > > > > > > > > > interval(not every second, defaults to 30secs).
> > > Storing
> > > > > > > remote
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> log
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> metadata in a strongly consistent
> > store
> > > > for
> > > > > > S3
> > > > > > > > RSM
> > > > > > > > > is
> > > > > > > > > > > > > raised
> > > > > > > > > > > > > > in PR-comment[1].
> > > > > > > > > > > > > > > > > >>>>>>> RLM invokes RSM at regular
> intervals
> > > and
> > > > RSM
> > > > > > > can
> > > > > > > > > give
> > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> segment
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> metadata if it is available. RSM is
> > > > > > responsible
> > > > > > > > for
> > > > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> fetching those entries. It should
> be
> > > > based on
> > > > > > > > > whatever
> > > > > > > > > > > > > > mechanism
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> consistent and efficient with the
> > > > respective
> > > > > > > > remote
> > > > > > > > > > > > > storage.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Can you give more details about
> push
> > > > based
> > > > > > > > > mechanism
> > > > > > > > > > > from
> > > > > > > > > > > > > > RSM?
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 1.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > >
> > > > > > > >
> > https://github.com/apache/kafka/pull/7561#discussion_r344576223
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > >>>>>>> Satish.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Thu, Dec 5, 2019 at 4:23 AM Jun
> > Rao
> > > <
> > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi, Harsha,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 40/41. I am curious which block
> > > storages
> > > > you
> > > > > > > have
> > > > > > > > > > > tested.
> > > > > > > > > > > > > S3
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> seems
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to be
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> one of the popular block stores.
> The
> > > > concerns
> > > > > > > > that
> > > > > > > > > I
> > > > > > > > > > > have
> > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> pull
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> based
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> approach are the following.
> > > > > > > > > > > > > > > > > >>>>>>> (a) Cost: S3 list object requests
> > cost
> > > > $0.005
> > > > > > > per
> > > > > > > > > 1000
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> requests. If
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> have 100,000 partitions and want to
> > > pull
> > > > the
> > > > > > > > > metadata
> > > > > > > > > > > for
> > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> partition
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> at
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the rate of 1/sec. It can cost
> > > $0.5/sec,
> > > > > > which
> > > > > > > is
> > > > > > > > > > > roughly
> > > > > > > > > > > > > > $40K
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> per
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> day.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (b) Semantics: S3 list objects are
> > > > eventually
> > > > > > > > > > > consistent.
> > > > > > > > > > > > > So,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> when
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> do a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> list object request, there is no
> > > > guarantee
> > > > > > that
> > > > > > > > > you can
> > > > > > > > > > > > see
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> uploaded
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> objects. This could impact the
> > > > correctness of
> > > > > > > > > > > subsequent
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> logics.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (c) Efficiency: Blindly pulling
> > > metadata
> > > > when
> > > > > > > > > there is
> > > > > > > > > > > no
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> change adds
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> unnecessary overhead in the broker
> as
> > > > well as
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > block
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> store.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> So, have you guys tested S3? If so,
> > > > could you
> > > > > > > > share
> > > > > > > > > > > your
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> experience
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> terms of cost, semantics and
> > > efficiency?
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Tue, Dec 3, 2019 at 10:11 PM
> > Harsha
> > > > > > > > > Chintalapani <
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> kafka@harsha.io
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 26, 2019 at 3:46 PM,
> Jun
> > > Rao
> > > > <
> > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Ying,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 40/41. There are two different ways
> > > that
> > > > we
> > > > > > can
> > > > > > > > > > > approach
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> this.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> One is
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> what
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> you said. We can have an
> opinionated
> > > way
> > > > of
> > > > > > > > > storing and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> populating
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> metadata that we think is good
> enough
> > > for
> > > > > > > > > everyone. I
> > > > > > > > > > > am
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> sure if
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> is the case based on what's
> currently
> > > > > > proposed
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> For
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> example, I
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> am not sure that (1) everyone
> always
> > > > needs
> > > > > > > local
> > > > > > > > > > > > metadata;
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (2)
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> current
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> local storage format is general
> > enough
> > > > and
> > > > > > (3)
> > > > > > > > > everyone
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wants to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> use
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> pull based approach to propagate
> the
> > > > > > metadata.
> > > > > > > > > Another
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> approach
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> is to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> make
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> this pluggable and let the
> > implementor
> > > > > > > implements
> > > > > > > > > the
> > > > > > > > > > > > best
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> approach
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> for a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> particular block storage. I haven't
> > > seen
> > > > any
> > > > > > > > > comments
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Slack/AirBnb
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the mailing list on this topic. It
> > > would
> > > > be
> > > > > > > great
> > > > > > > > > if
> > > > > > > > > > > they
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> provide
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> feedback directly here.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> The current interfaces are designed
> > > with
> > > > most
> > > > > > > > > popular
> > > > > > > > > > > > block
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storages
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> available today and we did 2
> > > > implementations
> > > > > > > with
> > > > > > > > > these
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> interfaces and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> they both are yielding good results
> > as
> > > we
> > > > > > going
> > > > > > > > > through
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> testing of
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> it.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> If there is ever a need for pull
> > based
> > > > > > approach
> > > > > > > > we
> > > > > > > > > can
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> definitely
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> evolve
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the interface.
> > > > > > > > > > > > > > > > > >>>>>>> In the past we did mark interfaces
> to
> > > be
> > > > > > > evolving
> > > > > > > > > to
> > > > > > > > > > > make
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> room for
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> unknowns
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> in the future.
> > > > > > > > > > > > > > > > > >>>>>>> If you have any suggestions around
> > the
> > > > > > current
> > > > > > > > > > > interfaces
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> please
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> propose we
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> are happy to see if we can work
> them
> > > > into it.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 43. To offer tier storage as a
> > general
> > > > > > feature,
> > > > > > > > > ideally
> > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> existing
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> capabilities should still be
> > supported.
> > > > It's
> > > > > > > fine
> > > > > > > > > if
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> uber
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> implementation doesn't support all
> > > > > > capabilities
> > > > > > > > for
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> internal
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> usage.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> However, the framework should be
> > > general
> > > > > > > enough.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> We agree on that as a principle.
> But
> > > all
> > > > of
> > > > > > > these
> > > > > > > > > major
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> features
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> mostly
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> coming right now and to have a new
> > big
> > > > > > feature
> > > > > > > > > such as
> > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> support all the new features will
> be
> > a
> > > > big
> > > > > > ask.
> > > > > > > > We
> > > > > > > > > can
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> document on
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> how
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> do
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> we approach solving these in future
> > > > > > iterations.
> > > > > > > > > > > > > > > > > >>>>>>> Our goal is to make this tiered
> > storage
> > > > > > feature
> > > > > > > > > work
> > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> everyone.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 43.3 This is more than just serving
> > the
> > > > > > tier-ed
> > > > > > > > > data
> > > > > > > > > > > from
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> block
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> With KIP-392, the consumer now can
> > > > resolve
> > > > > > the
> > > > > > > > > > > conflicts
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> with the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> replica
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> based on leader epoch. So, we need
> to
> > > > make
> > > > > > sure
> > > > > > > > > that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> leader epoch
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> can be
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> recovered properly from tier
> storage.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> We are working on testing our
> > approach
> > > > and we
> > > > > > > > will
> > > > > > > > > > > update
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the KIP
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> design details.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 43.4 For JBOD, if tier storage
> stores
> > > the
> > > > > > tier
> > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> locally, we
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> need to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> support moving such metadata across
> > > disk
> > > > > > > > > directories
> > > > > > > > > > > > since
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> JBOD
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> supports
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> moving data across disks.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> KIP is updated with JBOD details.
> > > Having
> > > > said
> > > > > > > > that
> > > > > > > > > JBOD
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> tooling
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> needs
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> evolve to support production loads.
> > > Most
> > > > of
> > > > > > the
> > > > > > > > > users
> > > > > > > > > > > > will
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> interested in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> using tiered storage without JBOD
> > > support
> > > > > > > support
> > > > > > > > > on
> > > > > > > > > > > day
> > > > > > > > > > > > 1.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > >>>>>>> Harsha
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> As for meeting, we could have a KIP
> > > > e-meeting
> > > > > > > on
> > > > > > > > > this
> > > > > > > > > > > if
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> needed,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> but it
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> will be open to everyone and will
> be
> > > > recorded
> > > > > > > and
> > > > > > > > > > > shared.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Often,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> details are still resolved through
> > the
> > > > > > mailing
> > > > > > > > > list.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 19, 2019 at 6:48 PM
> Ying
> > > > Zheng
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> <yi...@uber.com.invalid>
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Please ignore my previous email
> > > > > > > > > > > > > > > > > >>>>>>> I didn't know Apache requires all
> the
> > > > > > > discussions
> > > > > > > > > to be
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> "open"
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 19, 2019, 5:40 PM Ying
> > > Zheng
> > > > <
> > > > > > > > > > > yingz@uber.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thank you very much for your
> > feedback!
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Can we schedule a meeting in your
> > Palo
> > > > Alto
> > > > > > > > office
> > > > > > > > > in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> December? I
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> think a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> face to face discussion is much
> more
> > > > > > efficient
> > > > > > > > than
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> emails. Both
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Harsha
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> I can visit you. Satish may be able
> > to
> > > > join
> > > > > > us
> > > > > > > > > > > remotely.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Fri, Nov 15, 2019 at 11:04 AM
> Jun
> > > Rao
> > > > <
> > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Harsha,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> The following is a more detailed
> high
> > > > level
> > > > > > > > > feedback
> > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the KIP.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Overall,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the KIP seems useful. The challenge
> > is
> > > > how to
> > > > > > > > > design it
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> such that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> it’s
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> general enough to support different
> > > ways
> > > > of
> > > > > > > > > > > implementing
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> feature
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> support existing features.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 40. Local segment metadata storage:
> > The
> > > > KIP
> > > > > > > makes
> > > > > > > > > the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> assumption
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> metadata for the archived log
> > segments
> > > > are
> > > > > > > cached
> > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> every
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> broker
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> and provides a specific
> > implementation
> > > > for
> > > > > > the
> > > > > > > > > local
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> framework. We probably should
> discuss
> > > > this
> > > > > > > more.
> > > > > > > > > For
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> example,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> some
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage providers may not want to
> > cache
> > > > the
> > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> locally and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> just
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> rely
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> upon a remote key/value store if
> > such a
> > > > store
> > > > > > > is
> > > > > > > > > > > already
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> present. If
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> local store is used, there could be
> > > > different
> > > > > > > > ways
> > > > > > > > > of
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> implementing it
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (e.g., based on customized local
> > files,
> > > > an
> > > > > > > > embedded
> > > > > > > > > > > local
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> store
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> like
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> RocksDB, etc). An alternative of
> > > > designing
> > > > > > this
> > > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> provide an
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> interface for retrieving the tier
> > > segment
> > > > > > > > metadata
> > > > > > > > > and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> leave the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> details
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> of
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> how to get the metadata outside of
> > the
> > > > > > > framework.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 41. RemoteStorageManager interface
> > and
> > > > the
> > > > > > > usage
> > > > > > > > > of the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> interface in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> framework: I am not sure if the
> > > > interface is
> > > > > > > > > general
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> enough. For
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> example,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> it seems that RemoteLogIndexEntry
> is
> > > > tied to
> > > > > > a
> > > > > > > > > specific
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> way of
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storing
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> metadata in remote storage. The
> > > framework
> > > > > > uses
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> listRemoteSegments()
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> api
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> a pull based approach. However, in
> > some
> > > > other
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> implementations, a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> push
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> based
> > > > > > > > > > > > > > > > > >>>>>>> approach may be more preferred. I
> > don’t
> > > > have
> > > > > > a
> > > > > > > > > concrete
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> proposal
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> yet.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> But,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> it would be useful to give this
> area
> > > some
> > > > > > more
> > > > > > > > > thoughts
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> and see
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> if we
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> make the interface more general.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 42. In the diagram, the
> > > RemoteLogManager
> > > > is
> > > > > > > side
> > > > > > > > by
> > > > > > > > > > > side
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> LogManager.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> This KIP only discussed how the
> fetch
> > > > request
> > > > > > > is
> > > > > > > > > > > handled
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> between
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> two
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> layer. However, we should also
> > consider
> > > > how
> > > > > > > other
> > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> touch
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> log can be handled. e.g., list
> > offsets
> > > by
> > > > > > > > > timestamp,
> > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> records,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> etc.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Also, in this model, it's not clear
> > > which
> > > > > > > > > component is
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> responsible
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> for
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> managing the log start offset. It
> > seems
> > > > that
> > > > > > > the
> > > > > > > > > log
> > > > > > > > > > > > start
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> offset
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> could
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> be
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> changed by both RemoteLogManager
> and
> > > > > > > LogManager.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 43. There are quite a few existing
> > > > features
> > > > > > not
> > > > > > > > > covered
> > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> KIP.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> It
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> would be useful to discuss each of
> > > those.
> > > > > > > > > > > > > > > > > >>>>>>> 43.1 I won’t say that compacted
> > topics
> > > > are
> > > > > > > rarely
> > > > > > > > > used
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> always
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> small.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> For example, KStreams uses
> compacted
> > > > topics
> > > > > > for
> > > > > > > > > storing
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> states
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> sometimes the size of the topic
> could
> > > be
> > > > > > large.
> > > > > > > > > While
> > > > > > > > > > > it
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> might
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> be ok
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> support compacted topics initially,
> > it
> > > > would
> > > > > > be
> > > > > > > > > useful
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> have a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> high
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> level
> > > > > > > > > > > > > > > > > >>>>>>> idea on how this might be supported
> > > down
> > > > the
> > > > > > > road
> > > > > > > > > so
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> we
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> don’t
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> have
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> make incompatible API changes in
> the
> > > > future.
> > > > > > > > > > > > > > > > > >>>>>>> 43.2 We need to discuss how EOS is
> > > > supported.
> > > > > > > In
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> particular, how
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> producer state integrated with the
> > > remote
> > > > > > > > storage.
> > > > > > > > > 43.3
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Now that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> KIP-392
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (allow consumers to fetch from
> > closest
> > > > > > replica)
> > > > > > > > is
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> implemented,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> we
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> need
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> discuss how reading from a follower
> > > > replica
> > > > > > is
> > > > > > > > > > > supported
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> 43.4 We need to discuss how JBOD is
> > > > supported
> > > > > > > > with
> > > > > > > > > tier
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Fri, Nov 8, 2019 at 12:06 AM Tom
> > > > Bentley <
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> tbentley@redhat.com
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks for those insights Ying.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> On Thu, Nov 7, 2019 at 9:26 PM Ying
> > > Zheng
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> <yingz@uber.com.invalid
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Thanks, I missed that point.
> However,
> > > > there's
> > > > > > > > > still a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> point at
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> which
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> consumer fetches start getting
> served
> > > > from
> > > > > > > remote
> > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (even
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> if
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> point isn't as soon as the local
> log
> > > > > > retention
> > > > > > > > > > > > time/size).
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> This
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> represents
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> a kind of performance cliff edge
> and
> > > > what I'm
> > > > > > > > > really
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> interested
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> how
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> easy it is for a consumer which
> falls
> > > off
> > > > > > that
> > > > > > > > > cliff to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> catch up
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> and so
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> its
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> fetches again come from local
> > storage.
> > > > > > > Obviously
> > > > > > > > > this
> > > > > > > > > > > can
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> depend
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> on
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> all
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> sorts of factors (like production
> > rate,
> > > > > > > > consumption
> > > > > > > > > > > > rate),
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> so
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> it's
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> guaranteed (just like it's not
> > > > guaranteed for
> > > > > > > > Kafka
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> today), but
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> would
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> represent a new failure mode.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> As I have explained in the last
> mail,
> > > > it's a
> > > > > > > very
> > > > > > > > > rare
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> case that
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> a
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> consumer
> > > > > > > > > > > > > > > > > >>>>>>> need to read remote data. With our
> > > > experience
> > > > > > > at
> > > > > > > > > Uber,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> this only
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> happens
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> when the consumer service had an
> > outage
> > > > for
> > > > > > > > several
> > > > > > > > > > > > hours.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> There is not a "performance cliff"
> as
> > > you
> > > > > > > assume.
> > > > > > > > > The
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> even faster than local disks in
> terms
> > > of
> > > > > > > > bandwidth.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Reading from
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage is going to have higher
> > latency
> > > > than
> > > > > > > > local
> > > > > > > > > > > disk.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> But
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> since
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> consumer
> > > > > > > > > > > > > > > > > >>>>>>> is catching up several hours data,
> > it's
> > > > not
> > > > > > > > > sensitive
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> sub-second
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> level
> > > > > > > > > > > > > > > > > >>>>>>> latency, and each remote read
> request
> > > > will
> > > > > > > read a
> > > > > > > > > large
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> amount of
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> data to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> make the overall performance better
> > > than
> > > > > > > reading
> > > > > > > > > from
> > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> disks.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Another aspect I'd like to
> understand
> > > > better
> > > > > > is
> > > > > > > > the
> > > > > > > > > > > > effect
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> of
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> serving
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> fetch
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> request from remote storage has on
> > the
> > > > > > broker's
> > > > > > > > > network
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> utilization. If
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> we're just trimming the amount of
> > data
> > > > held
> > > > > > > > locally
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> (without
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> increasing
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> overall local+remote retention),
> then
> > > > we're
> > > > > > > > > effectively
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> trading
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> disk
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> bandwidth for network bandwidth
> when
> > > > serving
> > > > > > > > fetch
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> requests from
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage (which I understand to be a
> > > good
> > > > > > thing,
> > > > > > > > > since
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> brokers are
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> often/usually disk bound). But if
> > we're
> > > > > > > > increasing
> > > > > > > > > the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> overall
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> local+remote
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> retention then it's more likely
> that
> > > > network
> > > > > > > > itself
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> becomes the
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> bottleneck.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> I appreciate this is all rather
> hand
> > > > wavy,
> > > > > > I'm
> > > > > > > > just
> > > > > > > > > > > > trying
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> understand
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> how this would affect broker
> > > > performance, so
> > > > > > > I'd
> > > > > > > > be
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> grateful for
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> any
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> insights you can offer.
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> Network bandwidth is a function of
> > > > produce
> > > > > > > speed,
> > > > > > > > > it
> > > > > > > > > > > has
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> nothing
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> do
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> remote retention. As long as the
> data
> > > is
> > > > > > > shipped
> > > > > > > > to
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> storage,
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> keep the data there for 1 day or 1
> > year
> > > > or
> > > > > > 100
> > > > > > > > > years,
> > > > > > > > > > > it
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> doesn't
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> consume
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >>>>>>> any
> > > > > > > > > > > > > > > > > >>>>>>> network resources.
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
>

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

Posted by Kowshik Prakasam <kp...@confluent.io>.
Hi Harsha/Satish,

Hope you are doing well. Would you be able to please update the meeting
notes section for the most recent 2 meetings (from 10/13 and 11/10)? It
will be useful to share the context with the community.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes


Cheers,
Kowshik


On Tue, Nov 10, 2020 at 11:39 PM Kowshik Prakasam <kp...@confluent.io>
wrote:

> Hi Harsha,
>
> The goal we discussed is to aim for preview in AK 3.0. In order to get us
> there, it will be useful to think about the order in which the code changes
> will be implemented, reviewed and merged. Since you are driving the
> development, do you want to layout the order of things? For example, do you
> eventually want to break up the PR into multiple smaller ones? If so, you
> could list the milestones there. Another perspective is that this can be
> helpful to budget time suitably and to understand the progress.
> Let us know how we can help.
>
>
> Cheers,
> Kowshik
>
> On Tue, Nov 10, 2020 at 3:26 PM Harsha Chintalapani <ka...@harsha.io>
> wrote:
>
>> Thanks Kowshik for the link. Seems reasonable,  as we discussed on the
>> call, code and completion of this KIP will be taken up by us.
>> Regarding Milestone 2, what you think it needs to be clarified there?
>> I believe what we are promising in the KIP along with unit tests, systems
>> tests will be delivered and we can call that as preview.   We will be
>> running this in our production and continue to provide the data and
>> metrics
>> to push this feature to GA.
>>
>>
>>
>> On Tue, Nov 10, 2020 at 10:07 AM, Kowshik Prakasam <
>> kprakasam@confluent.io>
>> wrote:
>>
>> > Hi Harsha/Satish,
>> >
>> > Thanks for the discussion today. Here is a link to the KIP-405
>> <https://issues.apache.org/jira/browse/KIP-405> development
>> > milestones google doc we discussed in the meeting today: https://docs.
>> > google.com/document/d/1B5_jaZvWWb2DUpgbgImq0k_IPZ4DWrR8Ru7YpuJrXdc/edit
>> > . I have shared it with you. Please have a look and share your
>> > feedback/improvements. As we discussed, things are clear until
>> milestone 1.
>> > Beyond that, we can discuss it again (perhaps in next sync or later),
>> once
>> > you have thought through the implementation plan/milestones and release
>> > into preview in 3.0.
>> >
>> > Cheers,
>> > Kowshik
>> >
>> > On Tue, Nov 10, 2020 at 6:56 AM Satish Duggana <
>> satish.duggana@gmail.com>
>> > wrote:
>> >
>> > Hi Jun,
>> > Thanks for your comments. Please find the inline replies below.
>> >
>> > 605.2 "Build the local leader epoch cache by cutting the leader epoch
>> > sequence received from remote storage to [LSO, ELO]." I mentioned an
>> issue
>> > earlier. Suppose the leader's local start offset is 100. The follower
>> finds
>> > a remote segment covering offset range [80, 120). The producerState with
>> > this remote segment is up to offset 120. To trim the producerState to
>> > offset 100 requires more work since one needs to download the previous
>> > producerState up to offset 80 and then replay the messages from 80 to
>> 100.
>> > It seems that it's simpler in this case for the follower just to take
>> the
>> > remote segment as it is and start fetching from offset 120.
>> >
>> > We chose that approach to avoid any edge cases here. It may be possible
>> > that the remote log segment that is received may not have the same
>> leader
>> > epoch sequence from 100-120 as it contains on the leader(this can happen
>> > due to unclean leader). It is safe to start from what the leader returns
>> > here.Another way is to find the remote log segment
>> >
>> > 5016. Just to echo what Kowshik was saying. It seems that
>> > RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
>> > partition, not on the replicas for the __remote_log_segment_metadata
>> > partition. It's not clear how the leader of
>> __remote_log_segment_metadata
>> > obtains the metadata for remote segments for deletion.
>> >
>> > RLMM will always receive the callback for the remote log metadata topic
>> > partitions hosted on the local broker and these will be subscribed. I
>> will
>> > make this clear in the KIP.
>> >
>> > 5100. KIP-516 <https://issues.apache.org/jira/browse/KIP-516> has been
>> accepted and is being implemented now. Could you
>> > update the KIP based on topicID?
>> >
>> > We mentioned KIP-516 <https://issues.apache.org/jira/browse/KIP-516>
>> and how it helps. We will update this KIP with all
>> > the changes it brings with KIP-516
>> <https://issues.apache.org/jira/browse/KIP-516>.
>> >
>> > 5101. RLMM: It would be useful to clarify how the following two APIs are
>> > used. According to the wiki, the former is used for topic deletion and
>> the
>> > latter is used for retention. It seems that retention should use the
>> former
>> > since remote segments without a matching epoch in the leader
>> (potentially
>> > due to unclean leader election) also need to be garbage collected. The
>> > latter seems to be used for the new leader to determine the last tiered
>> > segment.
>> > default Iterator<RemoteLogSegmentMetadata>
>> > listRemoteLogSegments(TopicPartition topicPartition)
>> > Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
>> > topicPartition, long leaderEpoch);
>> >
>> > Right,.that is what we are currently doing. We will update the javadocs
>> > and wiki with that. Earlier, we did not want to remove the segments
>> which
>> > are not matched with leader epochs from the ladder partition as they
>> may be
>> > used later by a replica which can become a leader (unclean leader
>> election)
>> > and refer those segments. But that may leak these segments in remote
>> > storage until the topic lifetime. We decided to cleanup the segments
>> with
>> > the oldest incase of size based retention also.
>> >
>> > 5102. RSM:
>> > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
>> use
>> > RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
>> >
>> > It will be useful to have metadata for RSM to fetch log segment. It may
>> > create location/path using id with other metadata too.
>> >
>> > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
>> >
>> > Wanted to keep endPosition as optional to read till the end of the
>> segment
>> > and avoid sentinels.
>> >
>> > 5102.3 Why only some of the methods have default implementation and
>> others
>> > Don't?
>> >
>> > Actually, RSM will not have any default implementations. Those 3 methods
>> > were made default earlier for tests etc. Updated the wiki.
>> >
>> > 5102.4. Could we define RemoteLogSegmentMetadataUpdate and
>> > DeletePartitionUpdate?
>> >
>> > Sure, they will be added.
>> >
>> > 5102.5 LogSegmentData: It seems that it's easier to pass in
>> > leaderEpochIndex as a ByteBuffer or byte array than a file since it
>> will be
>> > generated in memory.
>> >
>> > Right, this is in plan.
>> >
>> > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
>> > and startOffset. For example, deleteRecords() could move the
>> startOffset to
>> > the middle of a segment. If we copy the full segment to remote storage,
>> the
>> > baseOffset and the startOffset will be different.
>> >
>> > Good point. startOffset is baseOffset by default, if not set explicitly.
>> >
>> > 5102.7 Could we define all the public methods for
>> RemoteLogSegmentMetadata
>> > and LogSegmentData?
>> >
>> > Sure, updated the wiki.
>> >
>> > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata
>> is
>> > inclusive/exclusive?
>> >
>> > It is inclusive, will update.
>> >
>> > 5103. configs:
>> > 5103.1 Could we define the default value of non-required configs (e.g
>> the
>> > size of new thread pools)?
>> >
>> > Sure, that makes sense.
>> >
>> > 5103.2 It seems that local.log.retention.ms should default to
>> retention.ms
>> > ,
>> > instead of remote.log.retention.minutes. Similarly, it seems that
>> > local.log.retention.bytes should default to segment.bytes.
>> >
>> > Right, we do not have remote.log.retention as we discussed earlier.
>> Thanks
>> > for catching the typo.
>> >
>> > 5103.3 remote.log.manager.thread.pool.size: The description says "used
>> in
>> > scheduling tasks to copy segments, fetch remote log indexes and clean up
>> > remote log segments". However, there is a separate config
>> > remote.log.reader.threads for fetching remote data. It's weird to fetch
>> > remote index and log in different thread pools since both are used for
>> > serving fetch requests.
>> >
>> > Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
>> > activities. Fetch path always goes through remote.log.reader.threads.
>> >
>> > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time
>> to
>> > back off when there is no work to do? If so, perhaps it can be renamed
>> as
>> > backoff.ms.
>> >
>> > This is the delay interval for each iteration. It may be renamed to
>> > remote.log.manager.task.delay.ms
>> >
>> > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
>> > so, they need to be listed in this section.
>> >
>> > remote.log.manager.task.interval.ms is the process internal, retry
>> > interval is missing in the configs, which will be updated in the KIP.
>> >
>> > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
>> > remote log segments to avoid multiple index fetches from the remote
>> > storage." Is the RLM in memory or on disk? If on disk, where is it
>> stored?
>> > Do we need a configuration to bound the size?
>> >
>> > It is stored on disk. They are stored in a directory
>> > `remote-log-index-cache` under log dir. We plan to have a config for
>> that
>> > instead of default. We will have a configuration for that.
>> >
>> > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
>> > different places. It would be useful to standardize the terminology.
>> >
>> > Sure.
>> >
>> > 5106. The section on "In BuildingRemoteLogAux state". It listed two
>> > options without saying which option is chosen.
>> > We already mentioned in the KIP that we chose option-2.
>> >
>> > 5107. Follower to leader transition: It has step 2, but not step 1.
>> Step-1
>> > is there but it is not explicitly highlighted. It is previous table to
>> > step-2.
>> >
>> > 5108. If a consumer fetches from the remote data and the remote storage
>> is
>> > not available, what error code is used in the fetch response?
>> >
>> > Good point. We have not yet defined the error for this case. We need to
>> > define an error message and send the same in fetch response.
>> >
>> > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
>> > offset whose timestamp is >= to the given timestamp in the request. That
>> > means it checks in remote log time indexes first, after which local log
>> > time indexes are checked." Could you document which method in RLMM is
>> used
>> > for this?
>> >
>> > Okay.
>> >
>> > 5110. Stopreplica: "it sets all the remote log segment metadata of that
>> > partition with a delete marker and publishes them to RLMM." This seems
>> > outdated given the new topic deletion logic.
>> >
>> > Will update with KIP-516
>> <https://issues.apache.org/jira/browse/KIP-516> related points.
>> >
>> > 5111. "RLM follower fetches the earliest offset for the earliest leader
>> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
>> int
>> > leaderEpoch) and updates that as the log start offset." Do we need that
>> > since replication propagates logStartOffset already?
>> >
>> > Good point. Right, existing replication protocol takes care of updating
>> > the followers’s log start offset received from the leader.
>> >
>> > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
>> > storage?
>> >
>> > Remote reads may fail within the current default wait time, but
>> subsequent
>> > fetches would be able to serve as that data is stored in the local
>> cache.
>> > This cache is currently implemented in RSMs. But we plan to pull this
>> into
>> > the remote log messaging layer in future.
>> >
>> > 5113. "Committed offsets can be stored in a local file to avoid reading
>> > the messages again when a broker is restarted." Could you describe the
>> > format and the location of the file? Also, could the same message be
>> > processed by RLMM again after broker restart? If so, how do we handle
>> that?
>> >
>> > Sure, we will update in the KIP.
>> >
>> > 5114. Message format
>> > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
>> > apiKey 0 and 1.
>> >
>> > Nice catch, that was a typo. Fixed in the wiki.
>> >
>> > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
>> endOffset
>> > is inclusive/exclusive?
>> > It is inclusive, will update.
>> >
>> > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
>> bit
>> > more? Is that the epoch of the leader when it copies the segment to
>> remote
>> > storage? Also, how will this field be used?
>> >
>> > Right, this is the leader epoch of the broker which copied this segment.
>> > This is helpful in reason about which broker copied the segment to
>> remote
>> > storage.
>> >
>> > 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
>> > Kafka already has a timestamp field. Could we just use that?
>> >
>> > This is the timestamp at which the respective event occurred. Added this
>> > to RemoteLogSegmentMetadata as RLMM can be any other implementation. We
>> > thought about that but it looked cleaner to use at the message structure
>> > level instead of getting that from the consumer record and using that to
>> > build the respective event.
>> >
>> > 5114.5 SegmentSizeInBytes: Could this just be int32?
>> >
>> > Right, it looks like config allows only int value >= 14.
>> >
>> > 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
>> > for compaction. Perhaps it can be renamed to sth like
>> > RemotePartitionRemover.
>> >
>> > I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we
>> have
>> > other manager classes like RLM, RLMM).
>> >
>> > 5116. "RLC receives the delete_partition_marked and processes it if it
>> is
>> > not yet processed earlier." How does it know whether
>> > delete_partition_marked has been processed earlier?
>> >
>> > This is to handle duplicate delete_partition_marked events. RLC
>> internally
>> > maintains a state for the delete_partition events and if it already has
>> an
>> > existing event then it ignores if it is already being processed.
>> >
>> > 5117. Should we add a new MessageFormatter to read the tier metadata
>> > topic?
>> >
>> > Right, this is in plan but did not mention it in the KIP. This will be
>> > useful for debugging purposes too.
>> >
>> > 5118. "Maximum remote log reader thread pool task queue size. If the
>> task
>> > queue is full, broker will stop reading remote log segments." What do we
>> > return to the fetch request in this case?
>> >
>> > We return an error response for that partition.
>> >
>> > 5119. It would be useful to list all things not supported in the first
>> > version in a Future work or Limitations section. For example, compacted
>> > topic, JBOD, changing remote.log.storage.enable from true to false, etc.
>> >
>> > We already have a non-goals section which is filled with some of these
>> > details. Do we need another limitations section?
>> >
>> > Thanks,
>> > Satish.
>> >
>> > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
>> >
>> > Hi, Satish,
>> >
>> > Thanks for the updated KIP. A few more comments below.
>> >
>> > 605.2 "Build the local leader epoch cache by cutting the leader epoch
>> > sequence received from remote storage to [LSO, ELO]." I mentioned an
>> >
>> > issue
>> >
>> > earlier. Suppose the leader's local start offset is 100. The follower
>> >
>> > finds
>> >
>> > a remote segment covering offset range [80, 120). The producerState with
>> > this remote segment is up to offset 120. To trim the producerState to
>> > offset 100 requires more work since one needs to download the previous
>> > producerState up to offset 80 and then replay the messages from 80 to
>> >
>> > 100.
>> >
>> > It seems that it's simpler in this case for the follower just to take
>> the
>> > remote segment as it is and start fetching from offset 120.
>> >
>> > 5016. Just to echo what Kowshik was saying. It seems that
>> > RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
>> > partition, not on the replicas for the __remote_log_segment_metadata
>> > partition. It's not clear how the leader of
>> __remote_log_segment_metadata
>> > obtains the metadata for remote segments for deletion.
>> >
>> > 5100. KIP-516 <https://issues.apache.org/jira/browse/KIP-516> has been
>> accepted and is being implemented now. Could you
>> > update the KIP based on topicID?
>> >
>> > 5101. RLMM: It would be useful to clarify how the following two APIs are
>> > used. According to the wiki, the former is used for topic deletion and
>> >
>> > the
>> >
>> > latter is used for retention. It seems that retention should use the
>> >
>> > former
>> >
>> > since remote segments without a matching epoch in the leader
>> (potentially
>> > due to unclean leader election) also need to be garbage collected. The
>> > latter seems to be used for the new leader to determine the last tiered
>> > segment.
>> > default Iterator<RemoteLogSegmentMetadata>
>> > listRemoteLogSegments(TopicPartition topicPartition)
>> > Iterator<RemoteLogSegmentMetadata>
>> >
>> > listRemoteLogSegments(TopicPartition
>> >
>> > topicPartition, long leaderEpoch);
>> >
>> > 5102. RSM:
>> > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
>> use
>> > RemoteLogSegmentId instead of RemoteLogSegmentMetadata. 5102.2 In
>> > fetchLogSegmentData(), should we use long instead of Long? 5102.3 Why
>> only
>> > some of the methods have default implementation and
>> >
>> > others
>> >
>> > don't?
>> > 5102.4. Could we define RemoteLogSegmentMetadataUpdate and
>> > DeletePartitionUpdate?
>> > 5102.5 LogSegmentData: It seems that it's easier to pass in
>> > leaderEpochIndex as a ByteBuffer or byte array than a file since it
>> >
>> > will
>> >
>> > be generated in memory.
>> > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
>> >
>> > and
>> >
>> > startOffset. For example, deleteRecords() could move the startOffset to
>> >
>> > the
>> >
>> > middle of a segment. If we copy the full segment to remote storage, the
>> > baseOffset and the startOffset will be different.
>> > 5102.7 Could we define all the public methods for
>> >
>> > RemoteLogSegmentMetadata
>> >
>> > and LogSegmentData?
>> > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata
>> is
>> > inclusive/exclusive?
>> >
>> > 5103. configs:
>> > 5103.1 Could we define the default value of non-required configs (e.g
>> the
>> > size of new thread pools)?
>> > 5103.2 It seems that local.log.retention.ms should default to
>> >
>> > retention.ms,
>> >
>> > instead of remote.log.retention.minutes. Similarly, it seems that
>> > local.log.retention.bytes should default to segment.bytes. 5103.3
>> > remote.log.manager.thread.pool.size: The description says "used in
>> > scheduling tasks to copy segments, fetch remote log indexes and clean up
>> > remote log segments". However, there is a separate config
>> > remote.log.reader.threads for fetching remote data. It's weird to fetch
>> > remote index and log in different thread pools since both are used for
>> > serving fetch requests.
>> > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time
>> >
>> > to
>> >
>> > back off when there is no work to do? If so, perhaps it can be renamed
>> as
>> > backoff.ms.
>> > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
>> > so, they need to be listed in this section.
>> >
>> > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
>> > remote log segments to avoid multiple index fetches from the remote
>> > storage." Is the RLM in memory or on disk? If on disk, where is it
>> >
>> > stored?
>> >
>> > Do we need a configuration to bound the size?
>> >
>> > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
>> > different places. It would be useful to standardize the terminology.
>> >
>> > 5106. The section on "In BuildingRemoteLogAux state". It listed two
>> >
>> > options
>> >
>> > without saying which option is chosen.
>> >
>> > 5107. Follower to leader transition: It has step 2, but not step 1.
>> >
>> > 5108. If a consumer fetches from the remote data and the remote storage
>> >
>> > is
>> >
>> > not available, what error code is used in the fetch response?
>> >
>> > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
>> > offset whose timestamp is >= to the given timestamp in the request. That
>> > means it checks in remote log time indexes first, after which local log
>> > time indexes are checked." Could you document which method in RLMM is
>> >
>> > used
>> >
>> > for this?
>> >
>> > 5110. Stopreplica: "it sets all the remote log segment metadata of that
>> > partition with a delete marker and publishes them to RLMM." This seems
>> > outdated given the new topic deletion logic.
>> >
>> > 5111. "RLM follower fetches the earliest offset for the earliest leader
>> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
>> >
>> > int
>> >
>> > leaderEpoch) and updates that as the log start offset." Do we need that
>> > since replication propagates logStartOffset already?
>> >
>> > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
>> > storage?
>> >
>> > 5113. "Committed offsets can be stored in a local file to avoid reading
>> >
>> > the
>> >
>> > messages again when a broker is restarted." Could you describe the
>> format
>> > and the location of the file? Also, could the same message be processed
>> >
>> > by
>> >
>> > RLMM again after broker restart? If so, how do we handle that?
>> >
>> > 5114. Message format
>> > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
>> > apiKey 0 and 1.
>> > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
>> >
>> > endOffset
>> >
>> > is inclusive/exclusive?
>> > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
>> >
>> > bit
>> >
>> > more? Is that the epoch of the leader when it copies the segment to
>> >
>> > remote
>> >
>> > storage? Also, how will this field be used?
>> > 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
>> > Kafka already has a timestamp field. Could we just use that? 5114.5
>> > SegmentSizeInBytes: Could this just be int32?
>> >
>> > 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
>> > for compaction. Perhaps it can be renamed to sth like
>> > RemotePartitionRemover.
>> >
>> > 5116. "RLC receives the delete_partition_marked and processes it if it
>> is
>> > not yet processed earlier." How does it know whether
>> > delete_partition_marked has been processed earlier?
>> >
>> > 5117. Should we add a new MessageFormatter to read the tier metadata
>> >
>> > topic?
>> >
>> > 5118. "Maximum remote log reader thread pool task queue size. If the
>> task
>> > queue is full, broker will stop reading remote log segments." What do we
>> > return to the fetch request in this case?
>> >
>> > 5119. It would be useful to list all things not supported in the first
>> > version in a Future work or Limitations section. For example, compacted
>> > topic, JBOD, changing remote.log.storage.enable from true to false, etc.
>> >
>> > Thanks,
>> >
>> > Jun
>> >
>> > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
>> kprakasam@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.duggana@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.duggana@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 <dhruvil@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 <ha...@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.duggana@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> <
>> 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>
>> >
>> > <https://issues.apache.org/jira/browse/LE-0> locally,
>> >
>> > but msg 5 LE-1 <https://issues.apache.org/jira/browse/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>
>> >
>> > <https://issues.apache.org/jira/browse/LE-0> offsets are [0, 4] and
>> >
>> > LE-2 <https://issues.apache.org/jira/browse/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
>> >
>> > (
>> >
>> > jun@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> <
>> 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>
>> >
>> > <https://issues.apache.org/jira/browse/LE-0> locally,
>> >
>> > but msg 5 LE-1 <https://issues.apache.org/jira/browse/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.duggana@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.ch@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> <
>> https://issues.apache.org/jira/browse/KIP-405>
>> >
>> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
>> >
>> > (
>> >
>> > https://cwiki.apache.org/confluence/display/KAFKA/
>> > KIP-405 <https://issues.apache.org/jira/browse/KIP-405>
>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
>> >
>> > )
>> >
>> > On Tue, Aug 25, 2020 at 1:03 PM Jun Rao < jun@
>> >
>> > confluent. io
>> >
>> > (
>> >
>> > jun@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 (
>> >
>> > kafka@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 (
>> >
>> > yingz@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.ch@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 ( kprakasam@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>
>> > <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.ch@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 ( ch@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.duggana@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
>> >
>> >
>>
>

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

Posted by Kowshik Prakasam <kp...@confluent.io>.
Hi Harsha,

The goal we discussed is to aim for preview in AK 3.0. In order to get us
there, it will be useful to think about the order in which the code changes
will be implemented, reviewed and merged. Since you are driving the
development, do you want to layout the order of things? For example, do you
eventually want to break up the PR into multiple smaller ones? If so, you
could list the milestones there. Another perspective is that this can be
helpful to budget time suitably and to understand the progress.
Let us know how we can help.


Cheers,
Kowshik

On Tue, Nov 10, 2020 at 3:26 PM Harsha Chintalapani <ka...@harsha.io> wrote:

> Thanks Kowshik for the link. Seems reasonable,  as we discussed on the
> call, code and completion of this KIP will be taken up by us.
> Regarding Milestone 2, what you think it needs to be clarified there?
> I believe what we are promising in the KIP along with unit tests, systems
> tests will be delivered and we can call that as preview.   We will be
> running this in our production and continue to provide the data and metrics
> to push this feature to GA.
>
>
>
> On Tue, Nov 10, 2020 at 10:07 AM, Kowshik Prakasam <kprakasam@confluent.io
> >
> wrote:
>
> > Hi Harsha/Satish,
> >
> > Thanks for the discussion today. Here is a link to the KIP-405
> <https://issues.apache.org/jira/browse/KIP-405> development
> > milestones google doc we discussed in the meeting today: https://docs.
> > google.com/document/d/1B5_jaZvWWb2DUpgbgImq0k_IPZ4DWrR8Ru7YpuJrXdc/edit
> > . I have shared it with you. Please have a look and share your
> > feedback/improvements. As we discussed, things are clear until milestone
> 1.
> > Beyond that, we can discuss it again (perhaps in next sync or later),
> once
> > you have thought through the implementation plan/milestones and release
> > into preview in 3.0.
> >
> > Cheers,
> > Kowshik
> >
> > On Tue, Nov 10, 2020 at 6:56 AM Satish Duggana <satish.duggana@gmail.com
> >
> > wrote:
> >
> > Hi Jun,
> > Thanks for your comments. Please find the inline replies below.
> >
> > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > sequence received from remote storage to [LSO, ELO]." I mentioned an
> issue
> > earlier. Suppose the leader's local start offset is 100. The follower
> finds
> > a remote segment covering offset range [80, 120). The producerState with
> > this remote segment is up to offset 120. To trim the producerState to
> > offset 100 requires more work since one needs to download the previous
> > producerState up to offset 80 and then replay the messages from 80 to
> 100.
> > It seems that it's simpler in this case for the follower just to take the
> > remote segment as it is and start fetching from offset 120.
> >
> > We chose that approach to avoid any edge cases here. It may be possible
> > that the remote log segment that is received may not have the same leader
> > epoch sequence from 100-120 as it contains on the leader(this can happen
> > due to unclean leader). It is safe to start from what the leader returns
> > here.Another way is to find the remote log segment
> >
> > 5016. Just to echo what Kowshik was saying. It seems that
> > RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> > partition, not on the replicas for the __remote_log_segment_metadata
> > partition. It's not clear how the leader of __remote_log_segment_metadata
> > obtains the metadata for remote segments for deletion.
> >
> > RLMM will always receive the callback for the remote log metadata topic
> > partitions hosted on the local broker and these will be subscribed. I
> will
> > make this clear in the KIP.
> >
> > 5100. KIP-516 <https://issues.apache.org/jira/browse/KIP-516> has been
> accepted and is being implemented now. Could you
> > update the KIP based on topicID?
> >
> > We mentioned KIP-516 <https://issues.apache.org/jira/browse/KIP-516>
> and how it helps. We will update this KIP with all
> > the changes it brings with KIP-516
> <https://issues.apache.org/jira/browse/KIP-516>.
> >
> > 5101. RLMM: It would be useful to clarify how the following two APIs are
> > used. According to the wiki, the former is used for topic deletion and
> the
> > latter is used for retention. It seems that retention should use the
> former
> > since remote segments without a matching epoch in the leader (potentially
> > due to unclean leader election) also need to be garbage collected. The
> > latter seems to be used for the new leader to determine the last tiered
> > segment.
> > default Iterator<RemoteLogSegmentMetadata>
> > listRemoteLogSegments(TopicPartition topicPartition)
> > Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
> > topicPartition, long leaderEpoch);
> >
> > Right,.that is what we are currently doing. We will update the javadocs
> > and wiki with that. Earlier, we did not want to remove the segments which
> > are not matched with leader epochs from the ladder partition as they may
> be
> > used later by a replica which can become a leader (unclean leader
> election)
> > and refer those segments. But that may leak these segments in remote
> > storage until the topic lifetime. We decided to cleanup the segments with
> > the oldest incase of size based retention also.
> >
> > 5102. RSM:
> > 5102.1 For methods like fetchLogSegmentData(), it seems that they can use
> > RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> >
> > It will be useful to have metadata for RSM to fetch log segment. It may
> > create location/path using id with other metadata too.
> >
> > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> >
> > Wanted to keep endPosition as optional to read till the end of the
> segment
> > and avoid sentinels.
> >
> > 5102.3 Why only some of the methods have default implementation and
> others
> > Don't?
> >
> > Actually, RSM will not have any default implementations. Those 3 methods
> > were made default earlier for tests etc. Updated the wiki.
> >
> > 5102.4. Could we define RemoteLogSegmentMetadataUpdate and
> > DeletePartitionUpdate?
> >
> > Sure, they will be added.
> >
> > 5102.5 LogSegmentData: It seems that it's easier to pass in
> > leaderEpochIndex as a ByteBuffer or byte array than a file since it will
> be
> > generated in memory.
> >
> > Right, this is in plan.
> >
> > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
> > and startOffset. For example, deleteRecords() could move the startOffset
> to
> > the middle of a segment. If we copy the full segment to remote storage,
> the
> > baseOffset and the startOffset will be different.
> >
> > Good point. startOffset is baseOffset by default, if not set explicitly.
> >
> > 5102.7 Could we define all the public methods for
> RemoteLogSegmentMetadata
> > and LogSegmentData?
> >
> > Sure, updated the wiki.
> >
> > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> > inclusive/exclusive?
> >
> > It is inclusive, will update.
> >
> > 5103. configs:
> > 5103.1 Could we define the default value of non-required configs (e.g the
> > size of new thread pools)?
> >
> > Sure, that makes sense.
> >
> > 5103.2 It seems that local.log.retention.ms should default to
> retention.ms
> > ,
> > instead of remote.log.retention.minutes. Similarly, it seems that
> > local.log.retention.bytes should default to segment.bytes.
> >
> > Right, we do not have remote.log.retention as we discussed earlier.
> Thanks
> > for catching the typo.
> >
> > 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> > scheduling tasks to copy segments, fetch remote log indexes and clean up
> > remote log segments". However, there is a separate config
> > remote.log.reader.threads for fetching remote data. It's weird to fetch
> > remote index and log in different thread pools since both are used for
> > serving fetch requests.
> >
> > Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
> > activities. Fetch path always goes through remote.log.reader.threads.
> >
> > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time
> to
> > back off when there is no work to do? If so, perhaps it can be renamed as
> > backoff.ms.
> >
> > This is the delay interval for each iteration. It may be renamed to
> > remote.log.manager.task.delay.ms
> >
> > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> > so, they need to be listed in this section.
> >
> > remote.log.manager.task.interval.ms is the process internal, retry
> > interval is missing in the configs, which will be updated in the KIP.
> >
> > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> > remote log segments to avoid multiple index fetches from the remote
> > storage." Is the RLM in memory or on disk? If on disk, where is it
> stored?
> > Do we need a configuration to bound the size?
> >
> > It is stored on disk. They are stored in a directory
> > `remote-log-index-cache` under log dir. We plan to have a config for that
> > instead of default. We will have a configuration for that.
> >
> > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> > different places. It would be useful to standardize the terminology.
> >
> > Sure.
> >
> > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> > options without saying which option is chosen.
> > We already mentioned in the KIP that we chose option-2.
> >
> > 5107. Follower to leader transition: It has step 2, but not step 1.
> Step-1
> > is there but it is not explicitly highlighted. It is previous table to
> > step-2.
> >
> > 5108. If a consumer fetches from the remote data and the remote storage
> is
> > not available, what error code is used in the fetch response?
> >
> > Good point. We have not yet defined the error for this case. We need to
> > define an error message and send the same in fetch response.
> >
> > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > offset whose timestamp is >= to the given timestamp in the request. That
> > means it checks in remote log time indexes first, after which local log
> > time indexes are checked." Could you document which method in RLMM is
> used
> > for this?
> >
> > Okay.
> >
> > 5110. Stopreplica: "it sets all the remote log segment metadata of that
> > partition with a delete marker and publishes them to RLMM." This seems
> > outdated given the new topic deletion logic.
> >
> > Will update with KIP-516 <https://issues.apache.org/jira/browse/KIP-516>
> related points.
> >
> > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> int
> > leaderEpoch) and updates that as the log start offset." Do we need that
> > since replication propagates logStartOffset already?
> >
> > Good point. Right, existing replication protocol takes care of updating
> > the followers’s log start offset received from the leader.
> >
> > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> > storage?
> >
> > Remote reads may fail within the current default wait time, but
> subsequent
> > fetches would be able to serve as that data is stored in the local cache.
> > This cache is currently implemented in RSMs. But we plan to pull this
> into
> > the remote log messaging layer in future.
> >
> > 5113. "Committed offsets can be stored in a local file to avoid reading
> > the messages again when a broker is restarted." Could you describe the
> > format and the location of the file? Also, could the same message be
> > processed by RLMM again after broker restart? If so, how do we handle
> that?
> >
> > Sure, we will update in the KIP.
> >
> > 5114. Message format
> > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> > apiKey 0 and 1.
> >
> > Nice catch, that was a typo. Fixed in the wiki.
> >
> > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> endOffset
> > is inclusive/exclusive?
> > It is inclusive, will update.
> >
> > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
> bit
> > more? Is that the epoch of the leader when it copies the segment to
> remote
> > storage? Also, how will this field be used?
> >
> > Right, this is the leader epoch of the broker which copied this segment.
> > This is helpful in reason about which broker copied the segment to remote
> > storage.
> >
> > 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> > Kafka already has a timestamp field. Could we just use that?
> >
> > This is the timestamp at which the respective event occurred. Added this
> > to RemoteLogSegmentMetadata as RLMM can be any other implementation. We
> > thought about that but it looked cleaner to use at the message structure
> > level instead of getting that from the consumer record and using that to
> > build the respective event.
> >
> > 5114.5 SegmentSizeInBytes: Could this just be int32?
> >
> > Right, it looks like config allows only int value >= 14.
> >
> > 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> > for compaction. Perhaps it can be renamed to sth like
> > RemotePartitionRemover.
> >
> > I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we have
> > other manager classes like RLM, RLMM).
> >
> > 5116. "RLC receives the delete_partition_marked and processes it if it is
> > not yet processed earlier." How does it know whether
> > delete_partition_marked has been processed earlier?
> >
> > This is to handle duplicate delete_partition_marked events. RLC
> internally
> > maintains a state for the delete_partition events and if it already has
> an
> > existing event then it ignores if it is already being processed.
> >
> > 5117. Should we add a new MessageFormatter to read the tier metadata
> > topic?
> >
> > Right, this is in plan but did not mention it in the KIP. This will be
> > useful for debugging purposes too.
> >
> > 5118. "Maximum remote log reader thread pool task queue size. If the task
> > queue is full, broker will stop reading remote log segments." What do we
> > return to the fetch request in this case?
> >
> > We return an error response for that partition.
> >
> > 5119. It would be useful to list all things not supported in the first
> > version in a Future work or Limitations section. For example, compacted
> > topic, JBOD, changing remote.log.storage.enable from true to false, etc.
> >
> > We already have a non-goals section which is filled with some of these
> > details. Do we need another limitations section?
> >
> > Thanks,
> > Satish.
> >
> > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
> >
> > Hi, Satish,
> >
> > Thanks for the updated KIP. A few more comments below.
> >
> > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > sequence received from remote storage to [LSO, ELO]." I mentioned an
> >
> > issue
> >
> > earlier. Suppose the leader's local start offset is 100. The follower
> >
> > finds
> >
> > a remote segment covering offset range [80, 120). The producerState with
> > this remote segment is up to offset 120. To trim the producerState to
> > offset 100 requires more work since one needs to download the previous
> > producerState up to offset 80 and then replay the messages from 80 to
> >
> > 100.
> >
> > It seems that it's simpler in this case for the follower just to take the
> > remote segment as it is and start fetching from offset 120.
> >
> > 5016. Just to echo what Kowshik was saying. It seems that
> > RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> > partition, not on the replicas for the __remote_log_segment_metadata
> > partition. It's not clear how the leader of __remote_log_segment_metadata
> > obtains the metadata for remote segments for deletion.
> >
> > 5100. KIP-516 <https://issues.apache.org/jira/browse/KIP-516> has been
> accepted and is being implemented now. Could you
> > update the KIP based on topicID?
> >
> > 5101. RLMM: It would be useful to clarify how the following two APIs are
> > used. According to the wiki, the former is used for topic deletion and
> >
> > the
> >
> > latter is used for retention. It seems that retention should use the
> >
> > former
> >
> > since remote segments without a matching epoch in the leader (potentially
> > due to unclean leader election) also need to be garbage collected. The
> > latter seems to be used for the new leader to determine the last tiered
> > segment.
> > default Iterator<RemoteLogSegmentMetadata>
> > listRemoteLogSegments(TopicPartition topicPartition)
> > Iterator<RemoteLogSegmentMetadata>
> >
> > listRemoteLogSegments(TopicPartition
> >
> > topicPartition, long leaderEpoch);
> >
> > 5102. RSM:
> > 5102.1 For methods like fetchLogSegmentData(), it seems that they can use
> > RemoteLogSegmentId instead of RemoteLogSegmentMetadata. 5102.2 In
> > fetchLogSegmentData(), should we use long instead of Long? 5102.3 Why
> only
> > some of the methods have default implementation and
> >
> > others
> >
> > don't?
> > 5102.4. Could we define RemoteLogSegmentMetadataUpdate and
> > DeletePartitionUpdate?
> > 5102.5 LogSegmentData: It seems that it's easier to pass in
> > leaderEpochIndex as a ByteBuffer or byte array than a file since it
> >
> > will
> >
> > be generated in memory.
> > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
> >
> > and
> >
> > startOffset. For example, deleteRecords() could move the startOffset to
> >
> > the
> >
> > middle of a segment. If we copy the full segment to remote storage, the
> > baseOffset and the startOffset will be different.
> > 5102.7 Could we define all the public methods for
> >
> > RemoteLogSegmentMetadata
> >
> > and LogSegmentData?
> > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> > inclusive/exclusive?
> >
> > 5103. configs:
> > 5103.1 Could we define the default value of non-required configs (e.g the
> > size of new thread pools)?
> > 5103.2 It seems that local.log.retention.ms should default to
> >
> > retention.ms,
> >
> > instead of remote.log.retention.minutes. Similarly, it seems that
> > local.log.retention.bytes should default to segment.bytes. 5103.3
> > remote.log.manager.thread.pool.size: The description says "used in
> > scheduling tasks to copy segments, fetch remote log indexes and clean up
> > remote log segments". However, there is a separate config
> > remote.log.reader.threads for fetching remote data. It's weird to fetch
> > remote index and log in different thread pools since both are used for
> > serving fetch requests.
> > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time
> >
> > to
> >
> > back off when there is no work to do? If so, perhaps it can be renamed as
> > backoff.ms.
> > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> > so, they need to be listed in this section.
> >
> > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> > remote log segments to avoid multiple index fetches from the remote
> > storage." Is the RLM in memory or on disk? If on disk, where is it
> >
> > stored?
> >
> > Do we need a configuration to bound the size?
> >
> > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> > different places. It would be useful to standardize the terminology.
> >
> > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> >
> > options
> >
> > without saying which option is chosen.
> >
> > 5107. Follower to leader transition: It has step 2, but not step 1.
> >
> > 5108. If a consumer fetches from the remote data and the remote storage
> >
> > is
> >
> > not available, what error code is used in the fetch response?
> >
> > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > offset whose timestamp is >= to the given timestamp in the request. That
> > means it checks in remote log time indexes first, after which local log
> > time indexes are checked." Could you document which method in RLMM is
> >
> > used
> >
> > for this?
> >
> > 5110. Stopreplica: "it sets all the remote log segment metadata of that
> > partition with a delete marker and publishes them to RLMM." This seems
> > outdated given the new topic deletion logic.
> >
> > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> >
> > int
> >
> > leaderEpoch) and updates that as the log start offset." Do we need that
> > since replication propagates logStartOffset already?
> >
> > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> > storage?
> >
> > 5113. "Committed offsets can be stored in a local file to avoid reading
> >
> > the
> >
> > messages again when a broker is restarted." Could you describe the format
> > and the location of the file? Also, could the same message be processed
> >
> > by
> >
> > RLMM again after broker restart? If so, how do we handle that?
> >
> > 5114. Message format
> > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> > apiKey 0 and 1.
> > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> >
> > endOffset
> >
> > is inclusive/exclusive?
> > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
> >
> > bit
> >
> > more? Is that the epoch of the leader when it copies the segment to
> >
> > remote
> >
> > storage? Also, how will this field be used?
> > 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> > Kafka already has a timestamp field. Could we just use that? 5114.5
> > SegmentSizeInBytes: Could this just be int32?
> >
> > 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> > for compaction. Perhaps it can be renamed to sth like
> > RemotePartitionRemover.
> >
> > 5116. "RLC receives the delete_partition_marked and processes it if it is
> > not yet processed earlier." How does it know whether
> > delete_partition_marked has been processed earlier?
> >
> > 5117. Should we add a new MessageFormatter to read the tier metadata
> >
> > topic?
> >
> > 5118. "Maximum remote log reader thread pool task queue size. If the task
> > queue is full, broker will stop reading remote log segments." What do we
> > return to the fetch request in this case?
> >
> > 5119. It would be useful to list all things not supported in the first
> > version in a Future work or Limitations section. For example, compacted
> > topic, JBOD, changing remote.log.storage.enable from true to false, etc.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <kprakasam@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.duggana@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.duggana@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 <dhruvil@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 <ha...@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.duggana@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> <
> 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>
> >
> > <https://issues.apache.org/jira/browse/LE-0> locally,
> >
> > but msg 5 LE-1 <https://issues.apache.org/jira/browse/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>
> >
> > <https://issues.apache.org/jira/browse/LE-0> offsets are [0, 4] and
> >
> > LE-2 <https://issues.apache.org/jira/browse/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
> >
> > (
> >
> > jun@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> <
> 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>
> >
> > <https://issues.apache.org/jira/browse/LE-0> locally,
> >
> > but msg 5 LE-1 <https://issues.apache.org/jira/browse/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.duggana@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.ch@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> <
> https://issues.apache.org/jira/browse/KIP-405>
> >
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
> >
> > (
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/
> > KIP-405 <https://issues.apache.org/jira/browse/KIP-405>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
> >
> > )
> >
> > On Tue, Aug 25, 2020 at 1:03 PM Jun Rao < jun@
> >
> > confluent. io
> >
> > (
> >
> > jun@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 (
> >
> > kafka@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 (
> >
> > yingz@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.ch@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 ( kprakasam@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>
> > <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.ch@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 ( ch@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.duggana@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
> >
> >
>

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

Posted by Harsha Chintalapani <ka...@harsha.io>.
Thanks Kowshik for the link. Seems reasonable,  as we discussed on the
call, code and completion of this KIP will be taken up by us.
Regarding Milestone 2, what you think it needs to be clarified there?
I believe what we are promising in the KIP along with unit tests, systems
tests will be delivered and we can call that as preview.   We will be
running this in our production and continue to provide the data and metrics
to push this feature to GA.



On Tue, Nov 10, 2020 at 10:07 AM, Kowshik Prakasam <kp...@confluent.io>
wrote:

> Hi Harsha/Satish,
>
> Thanks for the discussion today. Here is a link to the KIP-405 development
> milestones google doc we discussed in the meeting today: https://docs.
> google.com/document/d/1B5_jaZvWWb2DUpgbgImq0k_IPZ4DWrR8Ru7YpuJrXdc/edit
> . I have shared it with you. Please have a look and share your
> feedback/improvements. As we discussed, things are clear until milestone 1.
> Beyond that, we can discuss it again (perhaps in next sync or later), once
> you have thought through the implementation plan/milestones and release
> into preview in 3.0.
>
> Cheers,
> Kowshik
>
> On Tue, Nov 10, 2020 at 6:56 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> Hi Jun,
> Thanks for your comments. Please find the inline replies below.
>
> 605.2 "Build the local leader epoch cache by cutting the leader epoch
> sequence received from remote storage to [LSO, ELO]." I mentioned an issue
> earlier. Suppose the leader's local start offset is 100. The follower finds
> a remote segment covering offset range [80, 120). The producerState with
> this remote segment is up to offset 120. To trim the producerState to
> offset 100 requires more work since one needs to download the previous
> producerState up to offset 80 and then replay the messages from 80 to 100.
> It seems that it's simpler in this case for the follower just to take the
> remote segment as it is and start fetching from offset 120.
>
> We chose that approach to avoid any edge cases here. It may be possible
> that the remote log segment that is received may not have the same leader
> epoch sequence from 100-120 as it contains on the leader(this can happen
> due to unclean leader). It is safe to start from what the leader returns
> here.Another way is to find the remote log segment
>
> 5016. Just to echo what Kowshik was saying. It seems that
> RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> partition, not on the replicas for the __remote_log_segment_metadata
> partition. It's not clear how the leader of __remote_log_segment_metadata
> obtains the metadata for remote segments for deletion.
>
> RLMM will always receive the callback for the remote log metadata topic
> partitions hosted on the local broker and these will be subscribed. I will
> make this clear in the KIP.
>
> 5100. KIP-516 has been accepted and is being implemented now. Could you
> update the KIP based on topicID?
>
> We mentioned KIP-516 and how it helps. We will update this KIP with all
> the changes it brings with KIP-516.
>
> 5101. RLMM: It would be useful to clarify how the following two APIs are
> used. According to the wiki, the former is used for topic deletion and the
> latter is used for retention. It seems that retention should use the former
> since remote segments without a matching epoch in the leader (potentially
> due to unclean leader election) also need to be garbage collected. The
> latter seems to be used for the new leader to determine the last tiered
> segment.
> default Iterator<RemoteLogSegmentMetadata>
> listRemoteLogSegments(TopicPartition topicPartition)
> Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
> topicPartition, long leaderEpoch);
>
> Right,.that is what we are currently doing. We will update the javadocs
> and wiki with that. Earlier, we did not want to remove the segments which
> are not matched with leader epochs from the ladder partition as they may be
> used later by a replica which can become a leader (unclean leader election)
> and refer those segments. But that may leak these segments in remote
> storage until the topic lifetime. We decided to cleanup the segments with
> the oldest incase of size based retention also.
>
> 5102. RSM:
> 5102.1 For methods like fetchLogSegmentData(), it seems that they can use
> RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
>
> It will be useful to have metadata for RSM to fetch log segment. It may
> create location/path using id with other metadata too.
>
> 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
>
> Wanted to keep endPosition as optional to read till the end of the segment
> and avoid sentinels.
>
> 5102.3 Why only some of the methods have default implementation and others
> Don't?
>
> Actually, RSM will not have any default implementations. Those 3 methods
> were made default earlier for tests etc. Updated the wiki.
>
> 5102.4. Could we define RemoteLogSegmentMetadataUpdate and
> DeletePartitionUpdate?
>
> Sure, they will be added.
>
> 5102.5 LogSegmentData: It seems that it's easier to pass in
> leaderEpochIndex as a ByteBuffer or byte array than a file since it will be
> generated in memory.
>
> Right, this is in plan.
>
> 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
> and startOffset. For example, deleteRecords() could move the startOffset to
> the middle of a segment. If we copy the full segment to remote storage, the
> baseOffset and the startOffset will be different.
>
> Good point. startOffset is baseOffset by default, if not set explicitly.
>
> 5102.7 Could we define all the public methods for RemoteLogSegmentMetadata
> and LogSegmentData?
>
> Sure, updated the wiki.
>
> 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> inclusive/exclusive?
>
> It is inclusive, will update.
>
> 5103. configs:
> 5103.1 Could we define the default value of non-required configs (e.g the
> size of new thread pools)?
>
> Sure, that makes sense.
>
> 5103.2 It seems that local.log.retention.ms should default to retention.ms
> ,
> instead of remote.log.retention.minutes. Similarly, it seems that
> local.log.retention.bytes should default to segment.bytes.
>
> Right, we do not have remote.log.retention as we discussed earlier. Thanks
> for catching the typo.
>
> 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> scheduling tasks to copy segments, fetch remote log indexes and clean up
> remote log segments". However, there is a separate config
> remote.log.reader.threads for fetching remote data. It's weird to fetch
> remote index and log in different thread pools since both are used for
> serving fetch requests.
>
> Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
> activities. Fetch path always goes through remote.log.reader.threads.
>
> 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time to
> back off when there is no work to do? If so, perhaps it can be renamed as
> backoff.ms.
>
> This is the delay interval for each iteration. It may be renamed to
> remote.log.manager.task.delay.ms
>
> 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> so, they need to be listed in this section.
>
> remote.log.manager.task.interval.ms is the process internal, retry
> interval is missing in the configs, which will be updated in the KIP.
>
> 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> remote log segments to avoid multiple index fetches from the remote
> storage." Is the RLM in memory or on disk? If on disk, where is it stored?
> Do we need a configuration to bound the size?
>
> It is stored on disk. They are stored in a directory
> `remote-log-index-cache` under log dir. We plan to have a config for that
> instead of default. We will have a configuration for that.
>
> 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> different places. It would be useful to standardize the terminology.
>
> Sure.
>
> 5106. The section on "In BuildingRemoteLogAux state". It listed two
> options without saying which option is chosen.
> We already mentioned in the KIP that we chose option-2.
>
> 5107. Follower to leader transition: It has step 2, but not step 1. Step-1
> is there but it is not explicitly highlighted. It is previous table to
> step-2.
>
> 5108. If a consumer fetches from the remote data and the remote storage is
> not available, what error code is used in the fetch response?
>
> Good point. We have not yet defined the error for this case. We need to
> define an error message and send the same in fetch response.
>
> 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> offset whose timestamp is >= to the given timestamp in the request. That
> means it checks in remote log time indexes first, after which local log
> time indexes are checked." Could you document which method in RLMM is used
> for this?
>
> Okay.
>
> 5110. Stopreplica: "it sets all the remote log segment metadata of that
> partition with a delete marker and publishes them to RLMM." This seems
> outdated given the new topic deletion logic.
>
> Will update with KIP-516 related points.
>
> 5111. "RLM follower fetches the earliest offset for the earliest leader
> epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
> leaderEpoch) and updates that as the log start offset." Do we need that
> since replication propagates logStartOffset already?
>
> Good point. Right, existing replication protocol takes care of updating
> the followers’s log start offset received from the leader.
>
> 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> storage?
>
> Remote reads may fail within the current default wait time, but subsequent
> fetches would be able to serve as that data is stored in the local cache.
> This cache is currently implemented in RSMs. But we plan to pull this into
> the remote log messaging layer in future.
>
> 5113. "Committed offsets can be stored in a local file to avoid reading
> the messages again when a broker is restarted." Could you describe the
> format and the location of the file? Also, could the same message be
> processed by RLMM again after broker restart? If so, how do we handle that?
>
> Sure, we will update in the KIP.
>
> 5114. Message format
> 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> apiKey 0 and 1.
>
> Nice catch, that was a typo. Fixed in the wiki.
>
> 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether endOffset
> is inclusive/exclusive?
> It is inclusive, will update.
>
> 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a bit
> more? Is that the epoch of the leader when it copies the segment to remote
> storage? Also, how will this field be used?
>
> Right, this is the leader epoch of the broker which copied this segment.
> This is helpful in reason about which broker copied the segment to remote
> storage.
>
> 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> Kafka already has a timestamp field. Could we just use that?
>
> This is the timestamp at which the respective event occurred. Added this
> to RemoteLogSegmentMetadata as RLMM can be any other implementation. We
> thought about that but it looked cleaner to use at the message structure
> level instead of getting that from the consumer record and using that to
> build the respective event.
>
> 5114.5 SegmentSizeInBytes: Could this just be int32?
>
> Right, it looks like config allows only int value >= 14.
>
> 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> for compaction. Perhaps it can be renamed to sth like
> RemotePartitionRemover.
>
> I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we have
> other manager classes like RLM, RLMM).
>
> 5116. "RLC receives the delete_partition_marked and processes it if it is
> not yet processed earlier." How does it know whether
> delete_partition_marked has been processed earlier?
>
> This is to handle duplicate delete_partition_marked events. RLC internally
> maintains a state for the delete_partition events and if it already has an
> existing event then it ignores if it is already being processed.
>
> 5117. Should we add a new MessageFormatter to read the tier metadata
> topic?
>
> Right, this is in plan but did not mention it in the KIP. This will be
> useful for debugging purposes too.
>
> 5118. "Maximum remote log reader thread pool task queue size. If the task
> queue is full, broker will stop reading remote log segments." What do we
> return to the fetch request in this case?
>
> We return an error response for that partition.
>
> 5119. It would be useful to list all things not supported in the first
> version in a Future work or Limitations section. For example, compacted
> topic, JBOD, changing remote.log.storage.enable from true to false, etc.
>
> We already have a non-goals section which is filled with some of these
> details. Do we need another limitations section?
>
> Thanks,
> Satish.
>
> On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
>
> Hi, Satish,
>
> Thanks for the updated KIP. A few more comments below.
>
> 605.2 "Build the local leader epoch cache by cutting the leader epoch
> sequence received from remote storage to [LSO, ELO]." I mentioned an
>
> issue
>
> earlier. Suppose the leader's local start offset is 100. The follower
>
> finds
>
> a remote segment covering offset range [80, 120). The producerState with
> this remote segment is up to offset 120. To trim the producerState to
> offset 100 requires more work since one needs to download the previous
> producerState up to offset 80 and then replay the messages from 80 to
>
> 100.
>
> It seems that it's simpler in this case for the follower just to take the
> remote segment as it is and start fetching from offset 120.
>
> 5016. Just to echo what Kowshik was saying. It seems that
> RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> partition, not on the replicas for the __remote_log_segment_metadata
> partition. It's not clear how the leader of __remote_log_segment_metadata
> obtains the metadata for remote segments for deletion.
>
> 5100. KIP-516 has been accepted and is being implemented now. Could you
> update the KIP based on topicID?
>
> 5101. RLMM: It would be useful to clarify how the following two APIs are
> used. According to the wiki, the former is used for topic deletion and
>
> the
>
> latter is used for retention. It seems that retention should use the
>
> former
>
> since remote segments without a matching epoch in the leader (potentially
> due to unclean leader election) also need to be garbage collected. The
> latter seems to be used for the new leader to determine the last tiered
> segment.
> default Iterator<RemoteLogSegmentMetadata>
> listRemoteLogSegments(TopicPartition topicPartition)
> Iterator<RemoteLogSegmentMetadata>
>
> listRemoteLogSegments(TopicPartition
>
> topicPartition, long leaderEpoch);
>
> 5102. RSM:
> 5102.1 For methods like fetchLogSegmentData(), it seems that they can use
> RemoteLogSegmentId instead of RemoteLogSegmentMetadata. 5102.2 In
> fetchLogSegmentData(), should we use long instead of Long? 5102.3 Why only
> some of the methods have default implementation and
>
> others
>
> don't?
> 5102.4. Could we define RemoteLogSegmentMetadataUpdate and
> DeletePartitionUpdate?
> 5102.5 LogSegmentData: It seems that it's easier to pass in
> leaderEpochIndex as a ByteBuffer or byte array than a file since it
>
> will
>
> be generated in memory.
> 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
>
> and
>
> startOffset. For example, deleteRecords() could move the startOffset to
>
> the
>
> middle of a segment. If we copy the full segment to remote storage, the
> baseOffset and the startOffset will be different.
> 5102.7 Could we define all the public methods for
>
> RemoteLogSegmentMetadata
>
> and LogSegmentData?
> 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> inclusive/exclusive?
>
> 5103. configs:
> 5103.1 Could we define the default value of non-required configs (e.g the
> size of new thread pools)?
> 5103.2 It seems that local.log.retention.ms should default to
>
> retention.ms,
>
> instead of remote.log.retention.minutes. Similarly, it seems that
> local.log.retention.bytes should default to segment.bytes. 5103.3
> remote.log.manager.thread.pool.size: The description says "used in
> scheduling tasks to copy segments, fetch remote log indexes and clean up
> remote log segments". However, there is a separate config
> remote.log.reader.threads for fetching remote data. It's weird to fetch
> remote index and log in different thread pools since both are used for
> serving fetch requests.
> 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time
>
> to
>
> back off when there is no work to do? If so, perhaps it can be renamed as
> backoff.ms.
> 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> so, they need to be listed in this section.
>
> 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> remote log segments to avoid multiple index fetches from the remote
> storage." Is the RLM in memory or on disk? If on disk, where is it
>
> stored?
>
> Do we need a configuration to bound the size?
>
> 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> different places. It would be useful to standardize the terminology.
>
> 5106. The section on "In BuildingRemoteLogAux state". It listed two
>
> options
>
> without saying which option is chosen.
>
> 5107. Follower to leader transition: It has step 2, but not step 1.
>
> 5108. If a consumer fetches from the remote data and the remote storage
>
> is
>
> not available, what error code is used in the fetch response?
>
> 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> offset whose timestamp is >= to the given timestamp in the request. That
> means it checks in remote log time indexes first, after which local log
> time indexes are checked." Could you document which method in RLMM is
>
> used
>
> for this?
>
> 5110. Stopreplica: "it sets all the remote log segment metadata of that
> partition with a delete marker and publishes them to RLMM." This seems
> outdated given the new topic deletion logic.
>
> 5111. "RLM follower fetches the earliest offset for the earliest leader
> epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
>
> int
>
> leaderEpoch) and updates that as the log start offset." Do we need that
> since replication propagates logStartOffset already?
>
> 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> storage?
>
> 5113. "Committed offsets can be stored in a local file to avoid reading
>
> the
>
> messages again when a broker is restarted." Could you describe the format
> and the location of the file? Also, could the same message be processed
>
> by
>
> RLMM again after broker restart? If so, how do we handle that?
>
> 5114. Message format
> 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> apiKey 0 and 1.
> 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
>
> endOffset
>
> is inclusive/exclusive?
> 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
>
> bit
>
> more? Is that the epoch of the leader when it copies the segment to
>
> remote
>
> storage? Also, how will this field be used?
> 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> Kafka already has a timestamp field. Could we just use that? 5114.5
> SegmentSizeInBytes: Could this just be int32?
>
> 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> for compaction. Perhaps it can be renamed to sth like
> RemotePartitionRemover.
>
> 5116. "RLC receives the delete_partition_marked and processes it if it is
> not yet processed earlier." How does it know whether
> delete_partition_marked has been processed earlier?
>
> 5117. Should we add a new MessageFormatter to read the tier metadata
>
> topic?
>
> 5118. "Maximum remote log reader thread pool task queue size. If the task
> queue is full, broker will stop reading remote log segments." What do we
> return to the fetch request in this case?
>
> 5119. It would be useful to list all things not supported in the first
> version in a Future work or Limitations section. For example, compacted
> topic, JBOD, changing remote.log.storage.enable from true to false, etc.
>
> Thanks,
>
> Jun
>
> On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <kprakasam@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.duggana@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.duggana@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 <dhruvil@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 <ha...@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.duggana@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
>
> (
>
> jun@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.duggana@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.ch@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
>
> (
>
> jun@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 (
>
> kafka@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 (
>
> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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
>
>

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

Posted by Kowshik Prakasam <kp...@confluent.io>.
Hi Harsha/Satish,

Thanks for the discussion today. Here is a link to the KIP-405 development
milestones google doc we discussed in the meeting today:
https://docs.google.com/document/d/1B5_jaZvWWb2DUpgbgImq0k_IPZ4DWrR8Ru7YpuJrXdc/edit
. I have shared it with you. Please have a look and share your
feedback/improvements. As we discussed, things are clear until milestone 1.
Beyond that, we can discuss it again (perhaps in next sync or later), once
you have thought through the implementation plan/milestones and release
into preview in 3.0.


Cheers,
Kowshik


On Tue, Nov 10, 2020 at 6:56 AM Satish Duggana <sa...@gmail.com>
wrote:

> Hi Jun,
> Thanks for your comments. Please find the inline replies below.
>
> 605.2 "Build the local leader epoch cache by cutting the leader epoch
> sequence received from remote storage to [LSO, ELO]." I mentioned an issue
> earlier. Suppose the leader's local start offset is 100. The follower finds
> a remote segment covering offset range [80, 120). The producerState with
> this remote segment is up to offset 120. To trim the producerState to
> offset 100 requires more work since one needs to download the previous
> producerState up to offset 80 and then replay the messages from 80 to 100.
> It seems that it's simpler in this case for the follower just to take the
> remote segment as it is and start fetching from offset 120.
>
> We chose that approach to avoid any edge cases here. It may be
> possible that the remote log segment that is received may not have the
> same leader epoch sequence from 100-120 as it contains on the
> leader(this can happen due to unclean leader). It is safe to start
> from what the leader returns here.Another way is to find the remote
> log segment
>
> 5016. Just to echo what Kowshik was saying. It seems that
> RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> partition, not on the replicas for the __remote_log_segment_metadata
> partition. It's not clear how the leader of __remote_log_segment_metadata
> obtains the metadata for remote segments for deletion.
>
> RLMM will always receive the callback for the remote log metadata
> topic partitions hosted on the local broker and these will be
> subscribed. I will make this clear in the KIP.
>
> 5100. KIP-516 has been accepted and is being implemented now. Could you
> update the KIP based on topicID?
>
> We mentioned KIP-516 and how it helps. We will update this KIP with
> all the changes it brings with KIP-516.
>
> 5101. RLMM: It would be useful to clarify how the following two APIs are
> used. According to the wiki, the former is used for topic deletion and the
> latter is used for retention. It seems that retention should use the former
> since remote segments without a matching epoch in the leader (potentially
> due to unclean leader election) also need to be garbage collected. The
> latter seems to be used for the new leader to determine the last tiered
> segment.
>     default Iterator<RemoteLogSegmentMetadata>
> listRemoteLogSegments(TopicPartition topicPartition)
>     Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
> topicPartition, long leaderEpoch);
>
> Right,.that is what we are currently doing. We will update the
> javadocs and wiki with that. Earlier, we did not want to remove the
> segments which are not matched with leader epochs from the ladder
> partition as they may be used later by a replica which can become a
> leader (unclean leader election) and refer those segments. But that
> may leak these segments in remote storage until the topic lifetime. We
> decided to cleanup the segments with the oldest incase of size based
> retention also.
>
> 5102. RSM:
> 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
>
> It will be useful to have metadata for RSM to fetch log segment. It
> may create location/path using id with other metadata too.
>
> 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
>
> Wanted to keep endPosition as optional to read till the end of the
> segment and avoid sentinels.
>
> 5102.3 Why only some of the methods have default implementation and others
> Don't?
>
> Actually,  RSM will not have any default implementations. Those 3
> methods were made default earlier for tests etc. Updated the wiki.
>
> 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> and DeletePartitionUpdate?
>
> Sure, they will be added.
>
>
> 5102.5 LogSegmentData: It seems that it's easier to pass
> in leaderEpochIndex as a ByteBuffer or byte array than a file since it will
> be generated in memory.
>
> Right, this is in plan.
>
> 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset and
> startOffset. For example, deleteRecords() could move the startOffset to the
> middle of a segment. If we copy the full segment to remote storage, the
> baseOffset and the startOffset will be different.
>
> Good point. startOffset is baseOffset by default, if not set explicitly.
>
> 5102.7 Could we define all the public methods for RemoteLogSegmentMetadata
> and LogSegmentData?
>
> Sure, updated the wiki.
>
> 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> inclusive/exclusive?
>
> It is inclusive, will update.
>
> 5103. configs:
> 5103.1 Could we define the default value of non-required configs (e.g the
> size of new thread pools)?
>
> Sure, that makes sense.
>
> 5103.2 It seems that local.log.retention.ms should default to retention.ms
> ,
> instead of remote.log.retention.minutes. Similarly, it seems
> that local.log.retention.bytes should default to segment.bytes.
>
> Right, we do not have  remote.log.retention as we discussed earlier.
> Thanks for catching the typo.
>
> 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> scheduling tasks to copy segments, fetch remote log indexes and clean up
> remote log segments". However, there is a separate
> config remote.log.reader.threads for fetching remote data. It's weird to
> fetch remote index and log in different thread pools since both are used
> for serving fetch requests.
>
> Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
> activities. Fetch path always goes through remote.log.reader.threads.
>
> 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time to
> back off when there is no work to do? If so, perhaps it can be renamed as
> backoff.ms.
>
> This is the delay interval for each iteration. It may be renamed to
> remote.log.manager.task.delay.ms
>
> 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> so, they need to be listed in this section.
>
> remote.log.manager.task.interval.ms is the process internal, retry
> interval is missing in the configs, which will be updated in the KIP.
>
> 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> remote log segments to avoid multiple index fetches from the remote
> storage." Is the RLM in memory or on disk? If on disk, where is it stored?
> Do we need a configuration to bound the size?
>
> It is stored on disk. They are stored in a directory
> `remote-log-index-cache` under log dir. We plan to have a config for
> that instead of default. We will have a configuration for that.
>
> 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> different places. It would be useful to standardize the terminology.
>
> Sure.
>
> 5106. The section on "In BuildingRemoteLogAux state". It listed two options
> without saying which option is chosen.
> We already mentioned in the KIP that we chose option-2.
>
> 5107. Follower to leader transition: It has step 2, but not step 1.
> Step-1 is there but it is not explicitly highlighted. It is previous
> table to step-2.
>
> 5108. If a consumer fetches from the remote data and the remote storage is
> not available, what error code is used in the fetch response?
>
> Good point. We have not yet defined the error for this case. We need
> to define an error message and send the same in fetch response.
>
> 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> offset whose timestamp is >= to the given timestamp in the request. That
> means it checks in remote log time indexes first, after which local log
> time indexes are checked." Could you document which method in RLMM is used
> for this?
>
> Okay.
>
> 5110. Stopreplica: "it sets all the remote log segment metadata of that
> partition with a delete marker and publishes them to RLMM." This seems
> outdated given the new topic deletion logic.
>
> Will update with KIP-516 related points.
>
> 5111. "RLM follower fetches the earliest offset for the earliest leader
> epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
> leaderEpoch) and updates that as the log start offset." Do we need that
> since replication propagates logStartOffset already?
>
> Good point. Right, existing replication protocol takes care of
> updating the followers’s log start offset received from the leader.
>
> 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> storage?
>
> Remote reads may fail within the current default wait time, but
> subsequent fetches would be able to serve as that data is stored in
> the local cache. This cache is currently implemented in RSMs. But we
> plan to pull this into the remote log messaging layer in future.
>
> 5113. "Committed offsets can be stored in a local file to avoid reading the
> messages again when a broker is restarted." Could you describe the format
> and the location of the file? Also, could the same message be processed by
> RLMM again after broker restart? If so, how do we handle that?
>
> Sure, we will update in the KIP.
>
> 5114. Message format
> 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> apiKey 0 and 1.
>
> Nice catch, that was a typo. Fixed in the wiki.
>
> 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether endOffset
> is inclusive/exclusive?
> It is inclusive, will update.
>
> 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a bit
> more? Is that the epoch of the leader when it copies the segment to remote
> storage? Also, how will this field be used?
>
> Right, this is the leader epoch of the broker which copied this
> segment. This is helpful in reason about which broker copied the
> segment to remote storage.
>
> 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> Kafka already has a timestamp field. Could we just use that?
>
> This is the  timestamp at which  the respective event occurred. Added
> this  to RemoteLogSegmentMetadata as RLMM can be  any other
> implementation. We thought about that but it looked cleaner to use at
> the message structure level instead of getting that from the consumer
> record and using that to build the respective event.
>
>
> 5114.5 SegmentSizeInBytes: Could this just be int32?
>
> Right, it looks like config allows only int value >= 14.
>
> 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> for compaction. Perhaps it can be renamed to sth like
> RemotePartitionRemover.
>
> I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we
> have other manager classes like RLM, RLMM).
>
> 5116. "RLC receives the delete_partition_marked and processes it if it is
> not yet processed earlier." How does it know whether
> delete_partition_marked has been processed earlier?
>
> This is to handle duplicate delete_partition_marked events. RLC
> internally maintains a state for the delete_partition events and if it
> already has an existing event then it ignores if it is already being
> processed.
>
> 5117. Should we add a new MessageFormatter to read the tier metadata topic?
>
> Right, this is in plan but did not mention it in the KIP. This will be
> useful for debugging purposes too.
>
> 5118. "Maximum remote log reader thread pool task queue size. If the task
> queue is full, broker will stop reading remote log segments." What do we
> return to the fetch request in this case?
>
> We return an error response for that partition.
>
> 5119. It would be useful to list all things not supported in the first
> version in a Future work or Limitations section. For example, compacted
> topic, JBOD, changing remote.log.storage.enable from true to false, etc.
>
> We already have a non-goals section which is filled with some of these
> details. Do we need another limitations section?
>
> Thanks,
> Satish.
>
> On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
> >
> > Hi, Satish,
> >
> > Thanks for the updated KIP. A few more comments below.
> >
> > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > sequence received from remote storage to [LSO, ELO]." I mentioned an
> issue
> > earlier. Suppose the leader's local start offset is 100. The follower
> finds
> > a remote segment covering offset range [80, 120). The producerState with
> > this remote segment is up to offset 120. To trim the producerState to
> > offset 100 requires more work since one needs to download the previous
> > producerState up to offset 80 and then replay the messages from 80 to
> 100.
> > It seems that it's simpler in this case for the follower just to take the
> > remote segment as it is and start fetching from offset 120.
> >
> > 5016. Just to echo what Kowshik was saying. It seems that
> > RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> > partition, not on the replicas for the __remote_log_segment_metadata
> > partition. It's not clear how the leader of __remote_log_segment_metadata
> > obtains the metadata for remote segments for deletion.
> >
> > 5100. KIP-516 has been accepted and is being implemented now. Could you
> > update the KIP based on topicID?
> >
> > 5101. RLMM: It would be useful to clarify how the following two APIs are
> > used. According to the wiki, the former is used for topic deletion and
> the
> > latter is used for retention. It seems that retention should use the
> former
> > since remote segments without a matching epoch in the leader (potentially
> > due to unclean leader election) also need to be garbage collected. The
> > latter seems to be used for the new leader to determine the last tiered
> > segment.
> >     default Iterator<RemoteLogSegmentMetadata>
> > listRemoteLogSegments(TopicPartition topicPartition)
> >     Iterator<RemoteLogSegmentMetadata>
> listRemoteLogSegments(TopicPartition
> > topicPartition, long leaderEpoch);
> >
> > 5102. RSM:
> > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> > 5102.3 Why only some of the methods have default implementation and
> others
> > don't?
> > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > and DeletePartitionUpdate?
> > 5102.5 LogSegmentData: It seems that it's easier to pass
> > in leaderEpochIndex as a ByteBuffer or byte array than a file since it
> will
> > be generated in memory.
> > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
> and
> > startOffset. For example, deleteRecords() could move the startOffset to
> the
> > middle of a segment. If we copy the full segment to remote storage, the
> > baseOffset and the startOffset will be different.
> > 5102.7 Could we define all the public methods for
> RemoteLogSegmentMetadata
> > and LogSegmentData?
> > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> > inclusive/exclusive?
> >
> > 5103. configs:
> > 5103.1 Could we define the default value of non-required configs (e.g the
> > size of new thread pools)?
> > 5103.2 It seems that local.log.retention.ms should default to
> retention.ms,
> > instead of remote.log.retention.minutes. Similarly, it seems
> > that local.log.retention.bytes should default to segment.bytes.
> > 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> > scheduling tasks to copy segments, fetch remote log indexes and clean up
> > remote log segments". However, there is a separate
> > config remote.log.reader.threads for fetching remote data. It's weird to
> > fetch remote index and log in different thread pools since both are used
> > for serving fetch requests.
> > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time
> to
> > back off when there is no work to do? If so, perhaps it can be renamed as
> > backoff.ms.
> > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> > so, they need to be listed in this section.
> >
> > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> > remote log segments to avoid multiple index fetches from the remote
> > storage." Is the RLM in memory or on disk? If on disk, where is it
> stored?
> > Do we need a configuration to bound the size?
> >
> > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> > different places. It would be useful to standardize the terminology.
> >
> > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> options
> > without saying which option is chosen.
> >
> > 5107. Follower to leader transition: It has step 2, but not step 1.
> >
> > 5108. If a consumer fetches from the remote data and the remote storage
> is
> > not available, what error code is used in the fetch response?
> >
> > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > offset whose timestamp is >= to the given timestamp in the request. That
> > means it checks in remote log time indexes first, after which local log
> > time indexes are checked." Could you document which method in RLMM is
> used
> > for this?
> >
> > 5110. Stopreplica: "it sets all the remote log segment metadata of that
> > partition with a delete marker and publishes them to RLMM." This seems
> > outdated given the new topic deletion logic.
> >
> > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> int
> > leaderEpoch) and updates that as the log start offset." Do we need that
> > since replication propagates logStartOffset already?
> >
> > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> > storage?
> >
> > 5113. "Committed offsets can be stored in a local file to avoid reading
> the
> > messages again when a broker is restarted." Could you describe the format
> > and the location of the file? Also, could the same message be processed
> by
> > RLMM again after broker restart? If so, how do we handle that?
> >
> > 5114. Message format
> > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> > apiKey 0 and 1.
> > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> endOffset
> > is inclusive/exclusive?
> > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
> bit
> > more? Is that the epoch of the leader when it copies the segment to
> remote
> > storage? Also, how will this field be used?
> > 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> > Kafka already has a timestamp field. Could we just use that?
> > 5114.5 SegmentSizeInBytes: Could this just be int32?
> >
> > 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> > for compaction. Perhaps it can be renamed to sth like
> > RemotePartitionRemover.
> >
> > 5116. "RLC receives the delete_partition_marked and processes it if it is
> > not yet processed earlier." How does it know whether
> > delete_partition_marked has been processed earlier?
> >
> > 5117. Should we add a new MessageFormatter to read the tier metadata
> topic?
> >
> > 5118. "Maximum remote log reader thread pool task queue size. If the task
> > queue is full, broker will stop reading remote log segments." What do we
> > return to the fetch request in this case?
> >
> > 5119. It would be useful to list all things not supported in the first
> > version in a Future work or Limitations section. For example, compacted
> > topic, JBOD, changing remote.log.storage.enable from true to false, etc.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <kprakasam@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.duggana@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.duggana@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 <dhruvil@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 <ha...@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.duggana@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
> (
> > > > > > > > jun@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.duggana@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.ch@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
> > > (
> > > > > > > >>>> jun@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 (
> > > > > > > >>>>> kafka@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 (
> > > > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > >>>>>>>
> > > >
>

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

Posted by Israel Ekpo <is...@gmail.com>.
This is a very cool KIP.

It is going to have a significant impact on how Kafka is deployed and used
once it becomes implemented and available.

I am very interested in integrating/implementing the support for Azure Blob
Storage for this KIP as well. So far I have not seen any details about that
in the JIRA items.

I noticed there are recordings and written discussions, I would like to
participate in future discussions if possible.

Thanks for the great work.

On Mon, Jan 11, 2021 at 8:37 PM Jun Rao <ju...@confluent.io> wrote:

> Hi, Satish,
>
> Thanks for the reply. A few more followup comments.
>
> 6000. Since we are returning new error codes, we need to bump up the
> protocol version for Fetch request. Also, it will be useful to document all
> new error codes and whether they are retriable or not.
> This previous comment doesn't seem to be addressed.
>
> 9110. flat_file_format: Could you define the serialized representation for
> each field in the header?
>
> 9111. RPM has the following 2 steps.
> "1. The controller publishes delete_partition_marked event to say that the
> partition is marked for deletion. There can be multiple events published
> when the controller restarts or failover and this event will be
> deduplicated by RPM.
> 2. RPM receives the delete_partition_marked and processes it if it is not
> yet processed earlier."
> What triggers RPM to read __remote_log_metadata? Is RPM part of the default
> RLMM implementation or is it meant for any RLMM implementation?
>
> 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to make
> it clear in the comment that this is for the initial retry backoff.
>
> 9113. RLMM:
> 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be updated.
> 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> since all other update events have leaderEpoch?
> 9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
> to make it clear this is for deletion?
> 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
> clear this is for segment?
>
> 9114.Upgrade:
> 9114.1 It seems that we require message format > 0.11 for turning on the
> remote store feature.
> 9114.2 It's not clear to me why remote.log.storage.system.enable needs to
> be set to true before bumping up inter.broker.protocol.version.
> 9114.3 "If the topic-id is not received in the LeaderAndIsr request then
> remote log storage will not start. But it will log an error message in the
> log. One way to address this is to do a rolling restart of that broker, so
> that the leader will be moved to another broker and the controller will
> send LeaderAndIsr with the registered topic-id." Why is this needed? With
> KIP-516, after upgrading to the latest protocol, topicIds are auto created.
>
> 9115. testing: Could you comment on how we plan to do integration and
> system tests? Do we plan to include, for example, an in-memory
> implementation of RSM?
>
> 9116. There is still a todo for the message formatter.
>
> Jun
>
> On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi Jun,
> > Thanks for your comments. Please find the inline replies below.
> >
> > 6022. For packages used for server plugins, the convention is to
> > use org.apache.kafka.server. See java-based Authorizer as an example.
> >
> > Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> > ‘org.apache.kafka.server.log.remote.storage’.  Updated in the KIP.
> >
> > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat
> > file captures the state of the remote segments. After a partition is
> > deleted, it seems that we just need to remove the partitions's remote
> > segments from the flat file.
> >
> > DeletePartitionState might not yet have been processed by RPM and not
> > completed. We  will not have  that in flat  file format  once it
> > reaches DELETE_PARTITION_FINISHED state.
> >
> > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > version. It seems that's doable. One can just do the normal upgrade first
> > and wait enough time (for producer snapshots to be built), and then
> enable
> > remote storage.
> >
> > Upgrade notes updates in the KIP.
> >
> > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > tracking the remote segments when RLMM.onPartitionLeadershipChanges() is
> > called with the broker being the leader for __remote_log_metadata
> > partition? If so, could we document it?
> >
> > The current plan is to have that as part of RLMM and RPM uses that to
> > get the remote segments list. I will add this detail in the respective
> > sections.
> >
> > Satish.
> >
> > On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Hi, Satish,
> > >
> > > Thanks for the reply. A few more followup comments.
> > >
> > > 6022. For packages used for server plugins, the convention is to
> > > use org.apache.kafka.server. See java-based Authorizer as an example.
> > >
> > > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The
> flat
> > > file captures the state of the remote segments. After a partition is
> > > deleted, it seems that we just need to remove the partitions's remote
> > > segments from the flat file.
> > >
> > > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > > version. It seems that's doable. One can just do the normal upgrade
> first
> > > and wait enough time (for producer snapshots to be built), and then
> > enable
> > > remote storage.
> > >
> > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > > tracking the remote segments when RLMM.onPartitionLeadershipChanges()
> is
> > > called with the broker being the leader for __remote_log_metadata
> > > partition? If so, could we document it?
> > >
> > > Jun
> > >
> > > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <
> kprakasam@confluent.io
> > >
> > > wrote:
> > >
> > > > Hi Satish,
> > > >
> > > > Thanks for the updates! A few more comments below.
> > > >
> > > > 9001. Under the "Upgrade" section, there is a line mentioning:
> > "Upgrade the
> > > > existing Kafka cluster to 2.7 version and allow this to run for the
> log
> > > > retention of user topics that you want to enable tiered storage. This
> > will
> > > > allow all the topics to have the producer snapshots generated for
> each
> > log
> > > > segment." -- Which associated change in AK were you referring to
> here?
> > Is
> > > > it: https://github.com/apache/kafka/pull/7929 ? It seems like I
> don't
> > see
> > > > it in the 2.7 release branch yet, here is the link:
> > > > https://github.com/apache/kafka/commits/2.7.
> > > >
> > > > 9002. Under the "Upgrade" section, the configuration mentioned is
> > > > 'remote.log.storage.system.enable'. However, under "Public
> Interfaces"
> > > > section the corresponding configuration is
> > 'remote.storage.system.enable'.
> > > > Could we use the same one in both, maybe
> > > > 'remote.log.storage.system.enable'?
> > > >
> > > > 9003. Under "Per Topic Configuration", the KIP recommends setting
> > > > 'remote.log.storage.enable' to true at a per-topic level. It will be
> > useful
> > > > to add a line that if the user wants to enable it for all topics,
> then
> > they
> > > > should be able to set the cluster-wide default to true. Also, it will
> > be
> > > > useful to mention that the KIP currently does not support setting it
> to
> > > > false (after it is set to true), and add that to the future work
> > section.
> > > >
> > > > 9004. Under "Committed offsets file format", the sample provided
> shows
> > > > partition number and offset. Is the topic name required for
> identifying
> > > > which topic the partitions belong to?
> > > >
> > > > 9005. Under "Internal flat-file store format of remote log metadata",
> > it
> > > > seems useful to specify both topic name and topic ID for debugging
> > > > purposes.
> > > >
> > > > 9006. Under "Internal flat-file store format of remote log metadata",
> > the
> > > > description of "metadata-topic-offset" currently says "offset of the
> > remote
> > > > log metadata topic from which this topic partition's remote log
> > metadata is
> > > > fetched." Just for the wording, perhaps you meant to refer to the
> > offset
> > > > upto which the file has been committed? i.e. "offset of the remote
> log
> > > > metadata topic upto which this topic partition's remote log metadata
> > has
> > > > been committed into this file."
> > > >
> > > > 9007. Under "Internal flat-file store format of remote log metadata",
> > the
> > > > schema of the payload (i.e. beyond the header) seems to contain the
> > events
> > > > from the metadata topic. It seems useful to instead persist the
> > > > representation of the materialized state of the events, so that for
> the
> > > > same segment only the latest state is stored. Besides reducing
> storage
> > > > footprint, this also is likely to relate directly with the in-memory
> > > > representation of the RLMM cache (which probably is some kind of a
> Map
> > with
> > > > key being segment ID and value being the segment state), so recovery
> > from
> > > > disk will be straightforward.
> > > >
> > > > 9008. Under "Topic deletion lifecycle", step (1), it will be useful
> to
> > > > mention when in the deletion flow does the controller publish the
> > > > delete_partition_marked event to say that the partition is marked for
> > > > deletion?
> > > >
> > > > 9009. There are ~4 TODOs in the KIP. Could you please address these
> or
> > > > remove them?
> > > >
> > > > 9010. There is a reference to a Google doc on the KIP which was used
> > > > earlier for discussions. Please could you remove the reference, since
> > the
> > > > KIP is the source of the truth?
> > > >
> > > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > > >
> > > >
> > > > Cheers,
> > > > Kowshik
> > > >
> > > >
> > > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> > satish.duggana@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Jun,
> > > > > Thanks for your comments. Please go through the inline replies.
> > > > >
> > > > >
> > > > > 5102.2: It seems that both positions can just be int. Another
> option
> > is
> > > > to
> > > > > have two methods. Would it be clearer?
> > > > >
> > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata,  int startPosition)
> > > > throwsRemoteStorageException;
> > > > >
> > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata, int startPosition, int endPosition)
> throws
> > > > > RemoteStorageException;
> > > > >
> > > > > That makes sense to me, updated the KIP.
> > > > >
> > > > > 6003: Could you also update the javadoc for the return value?
> > > > >
> > > > > Updated.
> > > > >
> > > > > 6020: local.log.retention.bytes: Should it default to
> > log.retention.bytes
> > > > > to be consistent with local.log.retention.ms?
> > > > >
> > > > > Yes, it can be defaulted to log.retention.bytes.
> > > > >
> > > > > 6021: Could you define TopicIdPartition?
> > > > >
> > > > > Added TopicIdPartition in the KIP.
> > > > >
> > > > > 6022: For all public facing classes, could you specify the package
> > name?
> > > > >
> > > > > Updated.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > Satish.
> > > > >
> > > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > Hi, Satish,
> > > > > >
> > > > > > Thanks for the reply. A few more comments below.
> > > > > >
> > > > > > 5102.2: It seems that both positions can just be int. Another
> > option is
> > > > > to
> > > > > > have two methods. Would it be clearer?
> > > > > >
> > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata,
> > > > > >                                     int startPosition) throws
> > > > > > RemoteStorageException;
> > > > > >
> > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata,
> > > > > >                                     int startPosition, int
> > endPosition)
> > > > > > throws RemoteStorageException;
> > > > > >
> > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > >
> > > > > > 6010: What kind of tiering throughput have you seen with 5
> threads?
> > > > > >
> > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > log.retention.bytes
> > > > > > to be consistent with local.log.retention.ms?
> > > > > >
> > > > > > 6021: Could you define TopicIdPartition?
> > > > > >
> > > > > > 6022: For all public facing classes, could you specify the
> package
> > > > name?
> > > > > >
> > > > > > It seems that you already added the topicId support. Two other
> > > > remaining
> > > > > > items are (a) the format of local tier metadata storage and (b)
> > > > upgrade.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Jun,
> > > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > > >
> > > > > > > >605.2 It's rare for the follower to need the remote data. So,
> > the
> > > > > current
> > > > > > > approach is fine too. Could you document the process of
> > rebuilding
> > > > the
> > > > > > > producer state since we can't simply trim the producerState to
> an
> > > > > offset in
> > > > > > > the middle of a segment.
> > > > > > >
> > > > > > > Will clarify in the KIP.
> > > > > > >
> > > > > > > >5102.2 Would it be clearer to make startPosiont long and
> > endPosition
> > > > > of
> > > > > > > Optional<Long>?
> > > > > > >
> > > > > > > We will have arg checks with respective validation. It is not a
> > good
> > > > > > > practice to have arguments with optional as mentioned here.
> > > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > > >
> > > > > > >
> > > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as File
> > instead of
> > > > > > > ByteBuffer.
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >5102.7 Could you define all public methods for LogSegmentData?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >5103.5 Could you change the reference to
> > rlm_process_interval_ms and
> > > > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > > > interval
> > > > > > > config seems still missing. It would be useful to support
> > exponential
> > > > > > > backoff with the retry interval config.
> > > > > > >
> > > > > > > Good point. We wanted the retry with truncated exponential
> > backoff,
> > > > > > > updated the KIP.
> > > > > > >
> > > > > > > >5111. "RLM follower fetches the earliest offset for the
> earliest
> > > > > leader
> > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > topicPartition,
> > > > > int
> > > > > > > leaderEpoch) and updates that as the log start offset." This
> > text is
> > > > > still
> > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >5115. There are still references to "remote log cleaners".
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >6000. Since we are returning new error codes, we need to bump
> > up the
> > > > > > > protocol version for Fetch request. Also, it will be useful to
> > > > > document all
> > > > > > > new error codes and whether they are retriable or not.
> > > > > > >
> > > > > > > Sure, we will add that in the KIP.
> > > > > > >
> > > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > > > leaderEpoch
> > > > > > > is int32 instead of long.
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed
> > given
> > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > >
> > > > > > > No, it is fixed.
> > > > > > >
> > > > > > > >6003. RemoteLogSegmentMetadata
> > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > topicPartition, long offset, int epochForOffset): Should this
> > return
> > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > >
> > > > > > > That makes sense, updated.
> > > > > > >
> > > > > > > >6005. RemoteLogState: It seems it's better to split it between
> > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since
> > the
> > > > > states
> > > > > > > are never shared between the two use cases.
> > > > > > >
> > > > > > > Agree with that, updated.
> > > > > > >
> > > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > However,
> > > > > is it
> > > > > > > ture that other than the metadata topic, RLMM just needs to
> know
> > > > > whether
> > > > > > > there is a replica assigned to this broker and doesn't need to
> > know
> > > > > whether
> > > > > > > the replica is the leader or the follower?
> > > > > > >
> > > > > > > That may be true. If the implementation does not need that, it
> > can
> > > > > > > ignore the information in the callback.
> > > > > > >
> > > > > > > >6007: "Handle expired remote segments (leader and follower)":
> > Why is
> > > > > this
> > > > > > > needed in both the leader and the follower?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > > >                 "type": "int64",
> > > > > > > The segment size can just be int32.
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >6009. For the record format in the log, it seems that we need
> > to add
> > > > > > > record
> > > > > > > type and record version before the serialized bytes. We can
> > follow
> > > > the
> > > > > > > convention used in
> > > > > > >
> > > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > >
> > > > > > > Yes, KIP already mentions that these are serialized before the
> > > > payload
> > > > > > > as below. We will mention explicitly that these two are written
> > > > before
> > > > > > > the data is written.
> > > > > > >
> > > > > > > RLMM instance on broker publishes the message to the topic with
> > key
> > > > as
> > > > > > > null and value with the below format.
> > > > > > >
> > > > > > > type      : unsigned var int, represents the value type. This
> > value
> > > > is
> > > > > > > 'apikey' as mentioned in the schema.
> > > > > > > version : unsigned var int, the 'version' number of the type as
> > > > > > > mentioned in the schema.
> > > > > > > data      : record payload in kafka protocol message format.
> > > > > > >
> > > > > > >
> > > > > > > >6010. remote.log.manager.thread.pool.size: The default value
> is
> > 10.
> > > > > This
> > > > > > > might be too high when enabling the tiered feature for the
> first
> > > > time.
> > > > > > > Since there are lots of segments that need to be tiered
> > initially, a
> > > > > large
> > > > > > > number of threads could overwhelm the broker.
> > > > > > >
> > > > > > > Is the default value 5 reasonable?
> > > > > > >
> > > > > > > 6011. "The number of milli seconds to keep the local log
> segment
> > > > > before it
> > > > > > > gets deleted. If not set, the value in `log.retention.minutes`
> is
> > > > > used. If
> > > > > > > set to -1, no time limit is applied." We should use
> > log.retention.ms
> > > > > > > instead of log.retention.minutes.
> > > > > > > Nice typo catch. Updated the KIP.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > >
> > > > > > > > 605.2 It's rare for the follower to need the remote data. So,
> > the
> > > > > current
> > > > > > > > approach is fine too. Could you document the process of
> > rebuilding
> > > > > the
> > > > > > > > producer state since we can't simply trim the producerState
> to
> > an
> > > > > offset
> > > > > > > in
> > > > > > > > the middle of a segment.
> > > > > > > >
> > > > > > > > 5102.2 Would it be clearer to make startPosiont long and
> > > > endPosition
> > > > > of
> > > > > > > > Optional<Long>?
> > > > > > > >
> > > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File
> > instead of
> > > > > > > > ByteBuffer.
> > > > > > > >
> > > > > > > > 5102.7 Could you define all public methods for
> LogSegmentData?
> > > > > > > >
> > > > > > > > 5103.5 Could you change the reference to
> > rlm_process_interval_ms
> > > > and
> > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> retry
> > > > > interval
> > > > > > > > config seems still missing. It would be useful to support
> > > > exponential
> > > > > > > > backoff with the retry interval config.
> > > > > > > >
> > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > earliest
> > > > > leader
> > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > topicPartition,
> > > > > > > int
> > > > > > > > leaderEpoch) and updates that as the log start offset." This
> > text
> > > > is
> > > > > > > still
> > > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > > >
> > > > > > > > 5115. There are still references to "remote log cleaners".
> > > > > > > >
> > > > > > > > 6000. Since we are returning new error codes, we need to bump
> > up
> > > > the
> > > > > > > > protocol version for Fetch request. Also, it will be useful
> to
> > > > > document
> > > > > > > all
> > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > >
> > > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs():
> Currently,
> > > > > > > leaderEpoch
> > > > > > > > is int32 instead of long.
> > > > > > > >
> > > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed
> > given
> > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > >
> > > > > > > > 6003. RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > topicPartition, long offset, int epochForOffset): Should this
> > > > return
> > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > >
> > > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to
> > pick a
> > > > > more
> > > > > > > > indicative name so that people understand what epoch this is.
> > > > > > > >
> > > > > > > > 6005. RemoteLogState: It seems it's better to split it
> between
> > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> since
> > the
> > > > > states
> > > > > > > > are never shared between the two use cases.
> > > > > > > >
> > > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > However,
> > > > > is it
> > > > > > > > ture that other than the metadata topic, RLMM just needs to
> > know
> > > > > whether
> > > > > > > > there is a replica assigned to this broker and doesn't need
> to
> > know
> > > > > > > whether
> > > > > > > > the replica is the leader or the follower?
> > > > > > > >
> > > > > > > > 6007: "Handle expired remote segments (leader and follower)":
> > Why
> > > > is
> > > > > this
> > > > > > > > needed in both the leader and the follower?
> > > > > > > >
> > > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > > >                 "type": "int64",
> > > > > > > > The segment size can just be int32.
> > > > > > > >
> > > > > > > > 6009. For the record format in the log, it seems that we need
> > to
> > > > add
> > > > > > > record
> > > > > > > > type and record version before the serialized bytes. We can
> > follow
> > > > > the
> > > > > > > > convention used in
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > .
> > > > > > > >
> > > > > > > > 6010. remote.log.manager.thread.pool.size: The default value
> > is 10.
> > > > > This
> > > > > > > > might be too high when enabling the tiered feature for the
> > first
> > > > > time.
> > > > > > > > Since there are lots of segments that need to be tiered
> > initially,
> > > > a
> > > > > > > large
> > > > > > > > number of threads could overwhelm the broker.
> > > > > > > >
> > > > > > > > 6011. "The number of milli seconds to keep the local log
> > segment
> > > > > before
> > > > > > > it
> > > > > > > > gets deleted. If not set, the value in
> `log.retention.minutes`
> > is
> > > > > used.
> > > > > > > If
> > > > > > > > set to -1, no time limit is applied." We should use
> > > > log.retention.ms
> > > > > > > > instead of log.retention.minutes.
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > > satish.duggana@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi,
> > > > > > > > > We updated the KIP with the points mentioned in the earlier
> > mail
> > > > > > > > > except for KIP-516 related changes. You can go through them
> > and
> > > > > let us
> > > > > > > > > know if you have any comments. We will update the KIP with
> > the
> > > > > > > > > remaining todo items and KIP-516 related changes by end of
> > this
> > > > > > > > > week(5th Dec).
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Satish.
> > > > > > > > >
> > > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > > satish.duggana@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > Hi Jun,
> > > > > > > > > > Thanks for your comments. Please find the inline replies
> > below.
> > > > > > > > > >
> > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting the
> > leader
> > > > > epoch
> > > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > > > mentioned an
> > > > > > > > > issue
> > > > > > > > > > earlier. Suppose the leader's local start offset is 100.
> > The
> > > > > follower
> > > > > > > > > finds
> > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > producerState
> > > > > > > with
> > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > producerState to
> > > > > > > > > > offset 100 requires more work since one needs to download
> > the
> > > > > > > previous
> > > > > > > > > > producerState up to offset 80 and then replay the
> messages
> > from
> > > > > 80 to
> > > > > > > > > 100.
> > > > > > > > > > It seems that it's simpler in this case for the follower
> > just
> > > > to
> > > > > > > take the
> > > > > > > > > > remote segment as it is and start fetching from offset
> 120.
> > > > > > > > > >
> > > > > > > > > > We chose that approach to avoid any edge cases here. It
> > may be
> > > > > > > > > > possible that the remote log segment that is received may
> > not
> > > > > have
> > > > > > > the
> > > > > > > > > > same leader epoch sequence from 100-120 as it contains on
> > the
> > > > > > > > > > leader(this can happen due to unclean leader). It is safe
> > to
> > > > > start
> > > > > > > > > > from what the leader returns here.Another way is to find
> > the
> > > > > remote
> > > > > > > > > > log segment
> > > > > > > > > >
> > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > > > > replicas
> > > > > > > for a
> > > > > > > > > > partition, not on the replicas for the
> > > > > __remote_log_segment_metadata
> > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > __remote_log_segment_metadata
> > > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > > >
> > > > > > > > > > RLMM will always receive the callback for the remote log
> > > > metadata
> > > > > > > > > > topic partitions hosted on the local broker and these
> will
> > be
> > > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > > >
> > > > > > > > > > 5100. KIP-516 has been accepted and is being implemented
> > now.
> > > > > Could
> > > > > > > you
> > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > >
> > > > > > > > > > We mentioned KIP-516 and how it helps. We will update
> this
> > KIP
> > > > > with
> > > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > > >
> > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > following two
> > > > > APIs
> > > > > > > are
> > > > > > > > > > used. According to the wiki, the former is used for topic
> > > > > deletion
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > latter is used for retention. It seems that retention
> > should
> > > > use
> > > > > the
> > > > > > > > > former
> > > > > > > > > > since remote segments without a matching epoch in the
> > leader
> > > > > > > (potentially
> > > > > > > > > > due to unclean leader election) also need to be garbage
> > > > > collected.
> > > > > > > The
> > > > > > > > > > latter seems to be used for the new leader to determine
> the
> > > > last
> > > > > > > tiered
> > > > > > > > > > segment.
> > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > >
> > > > > > > > > > Right,.that is what we are currently doing. We will
> update
> > the
> > > > > > > > > > javadocs and wiki with that. Earlier, we did not want to
> > remove
> > > > > the
> > > > > > > > > > segments which are not matched with leader epochs from
> the
> > > > ladder
> > > > > > > > > > partition as they may be used later by a replica which
> can
> > > > > become a
> > > > > > > > > > leader (unclean leader election) and refer those
> segments.
> > But
> > > > > that
> > > > > > > > > > may leak these segments in remote storage until the topic
> > > > > lifetime.
> > > > > > > We
> > > > > > > > > > decided to cleanup the segments with the oldest incase of
> > size
> > > > > based
> > > > > > > > > > retention also.
> > > > > > > > > >
> > > > > > > > > > 5102. RSM:
> > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems
> > that
> > > > > they can
> > > > > > > > > > use RemoteLogSegmentId instead of
> RemoteLogSegmentMetadata.
> > > > > > > > > >
> > > > > > > > > > It will be useful to have metadata for RSM to fetch log
> > > > segment.
> > > > > It
> > > > > > > > > > may create location/path using id with other metadata
> too.
> > > > > > > > > >
> > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > instead of
> > > > > Long?
> > > > > > > > > >
> > > > > > > > > > Wanted to keep endPosition as optional to read till the
> > end of
> > > > > the
> > > > > > > > > > segment and avoid sentinels.
> > > > > > > > > >
> > > > > > > > > > 5102.3 Why only some of the methods have default
> > implementation
> > > > > and
> > > > > > > > > others
> > > > > > > > > > Don't?
> > > > > > > > > >
> > > > > > > > > > Actually,  RSM will not have any default implementations.
> > > > Those 3
> > > > > > > > > > methods were made default earlier for tests etc. Updated
> > the
> > > > > wiki.
> > > > > > > > > >
> > > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > >
> > > > > > > > > > Sure, they will be added.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a
> > file
> > > > > since
> > > > > > > it
> > > > > > > > > will
> > > > > > > > > > be generated in memory.
> > > > > > > > > >
> > > > > > > > > > Right, this is in plan.
> > > > > > > > > >
> > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs
> > both
> > > > > > > baseOffset
> > > > > > > > > and
> > > > > > > > > > startOffset. For example, deleteRecords() could move the
> > > > > startOffset
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > middle of a segment. If we copy the full segment to
> remote
> > > > > storage,
> > > > > > > the
> > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > >
> > > > > > > > > > Good point. startOffset is baseOffset by default, if not
> > set
> > > > > > > explicitly.
> > > > > > > > > >
> > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > and LogSegmentData?
> > > > > > > > > >
> > > > > > > > > > Sure, updated the wiki.
> > > > > > > > > >
> > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > > inclusive/exclusive?
> > > > > > > > > >
> > > > > > > > > > It is inclusive, will update.
> > > > > > > > > >
> > > > > > > > > > 5103. configs:
> > > > > > > > > > 5103.1 Could we define the default value of non-required
> > > > configs
> > > > > > > (e.g the
> > > > > > > > > > size of new thread pools)?
> > > > > > > > > >
> > > > > > > > > > Sure, that makes sense.
> > > > > > > > > >
> > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > default to
> > > > > > > > > retention.ms,
> > > > > > > > > > instead of remote.log.retention.minutes. Similarly, it
> > seems
> > > > > > > > > > that local.log.retention.bytes should default to
> > segment.bytes.
> > > > > > > > > >
> > > > > > > > > > Right, we do not have  remote.log.retention as we
> discussed
> > > > > earlier.
> > > > > > > > > > Thanks for catching the typo.
> > > > > > > > > >
> > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> description
> > > > says
> > > > > > > "used in
> > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > indexes and
> > > > > > > clean up
> > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > config remote.log.reader.threads for fetching remote
> data.
> > It's
> > > > > > > weird to
> > > > > > > > > > fetch remote index and log in different thread pools
> since
> > both
> > > > > are
> > > > > > > used
> > > > > > > > > > for serving fetch requests.
> > > > > > > > > >
> > > > > > > > > > Right, remote.log.manager.thread.pool is mainly used for
> > > > > copy/cleanup
> > > > > > > > > > activities. Fetch path always goes through
> > > > > remote.log.reader.threads.
> > > > > > > > > >
> > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> > amount
> > > > > of
> > > > > > > time
> > > > > > > > > to
> > > > > > > > > > back off when there is no work to do? If so, perhaps it
> > can be
> > > > > > > renamed as
> > > > > > > > > > backoff.ms.
> > > > > > > > > >
> > > > > > > > > > This is the delay interval for each iteration. It may be
> > > > renamed
> > > > > to
> > > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > > >
> > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > rlm_retry_interval_ms
> > > > > > > configs? If
> > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > >
> > > > > > > > > > remote.log.manager.task.interval.ms is the process
> > internal,
> > > > > retry
> > > > > > > > > > interval is missing in the configs, which will be updated
> > in
> > > > the
> > > > > KIP.
> > > > > > > > > >
> > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the
> > index
> > > > > > > files of
> > > > > > > > > > remote log segments to avoid multiple index fetches from
> > the
> > > > > remote
> > > > > > > > > > storage." Is the RLM in memory or on disk? If on disk,
> > where is
> > > > > it
> > > > > > > > > stored?
> > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > >
> > > > > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > > > > `remote-log-index-cache` under log dir. We plan to have a
> > > > config
> > > > > for
> > > > > > > > > > that instead of default. We will have a configuration for
> > that.
> > > > > > > > > >
> > > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest
> > Local
> > > > > Offset
> > > > > > > in
> > > > > > > > > > different places. It would be useful to standardize the
> > > > > terminology.
> > > > > > > > > >
> > > > > > > > > > Sure.
> > > > > > > > > >
> > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> > listed
> > > > > two
> > > > > > > > > options
> > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > > > > > >
> > > > > > > > > > 5107. Follower to leader transition: It has step 2, but
> not
> > > > step
> > > > > 1.
> > > > > > > > > > Step-1 is there but it is not explicitly highlighted. It
> is
> > > > > previous
> > > > > > > > > > table to step-2.
> > > > > > > > > >
> > > > > > > > > > 5108. If a consumer fetches from the remote data and the
> > remote
> > > > > > > storage
> > > > > > > > > is
> > > > > > > > > > not available, what error code is used in the fetch
> > response?
> > > > > > > > > >
> > > > > > > > > > Good point. We have not yet defined the error for this
> > case. We
> > > > > need
> > > > > > > > > > to define an error message and send the same in fetch
> > response.
> > > > > > > > > >
> > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the
> > first
> > > > > message
> > > > > > > > > > offset whose timestamp is >= to the given timestamp in
> the
> > > > > request.
> > > > > > > That
> > > > > > > > > > means it checks in remote log time indexes first, after
> > which
> > > > > local
> > > > > > > log
> > > > > > > > > > time indexes are checked." Could you document which
> method
> > in
> > > > > RLMM is
> > > > > > > > > used
> > > > > > > > > > for this?
> > > > > > > > > >
> > > > > > > > > > Okay.
> > > > > > > > > >
> > > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > metadata
> > > > > of
> > > > > > > that
> > > > > > > > > > partition with a delete marker and publishes them to
> RLMM."
> > > > This
> > > > > > > seems
> > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > >
> > > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > > >
> > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > earliest
> > > > > > > leader
> > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > > int
> > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> Do
> > we
> > > > > need
> > > > > > > that
> > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > >
> > > > > > > > > > Good point. Right, existing replication protocol takes
> > care of
> > > > > > > > > > updating the followers’s log start offset received from
> the
> > > > > leader.
> > > > > > > > > >
> > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> fetching
> > > > from
> > > > > > > remote
> > > > > > > > > > storage?
> > > > > > > > > >
> > > > > > > > > > Remote reads may fail within the current default wait
> > time, but
> > > > > > > > > > subsequent fetches would be able to serve as that data is
> > > > stored
> > > > > in
> > > > > > > > > > the local cache. This cache is currently implemented in
> > RSMs.
> > > > > But we
> > > > > > > > > > plan to pull this into the remote log messaging layer in
> > > > future.
> > > > > > > > > >
> > > > > > > > > > 5113. "Committed offsets can be stored in a local file to
> > avoid
> > > > > > > reading
> > > > > > > > > the
> > > > > > > > > > messages again when a broker is restarted." Could you
> > describe
> > > > > the
> > > > > > > format
> > > > > > > > > > and the location of the file? Also, could the same
> message
> > be
> > > > > > > processed
> > > > > > > > > by
> > > > > > > > > > RLMM again after broker restart? If so, how do we handle
> > that?
> > > > > > > > > >
> > > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > > >
> > > > > > > > > > 5114. Message format
> > > > > > > > > > 5114.1 There are two records named
> > > > RemoteLogSegmentMetadataRecord
> > > > > > > with
> > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > >
> > > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > > >
> > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > > > whether
> > > > > > > > > endOffset
> > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > It is inclusive, will update.
> > > > > > > > > >
> > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > > > LeaderEpoch
> > > > > > > a
> > > > > > > > > bit
> > > > > > > > > > more? Is that the epoch of the leader when it copies the
> > > > segment
> > > > > to
> > > > > > > > > remote
> > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > >
> > > > > > > > > > Right, this is the leader epoch of the broker which
> copied
> > this
> > > > > > > > > > segment. This is helpful in reason about which broker
> > copied
> > > > the
> > > > > > > > > > segment to remote storage.
> > > > > > > > > >
> > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more?
> > Each
> > > > > > > record in
> > > > > > > > > > Kafka already has a timestamp field. Could we just use
> > that?
> > > > > > > > > >
> > > > > > > > > > This is the  timestamp at which  the respective event
> > occurred.
> > > > > Added
> > > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any
> other
> > > > > > > > > > implementation. We thought about that but it looked
> > cleaner to
> > > > > use at
> > > > > > > > > > the message structure level instead of getting that from
> > the
> > > > > consumer
> > > > > > > > > > record and using that to build the respective event.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > >
> > > > > > > > > > Right, it looks like config allows only int value >= 14.
> > > > > > > > > >
> > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with
> > the
> > > > log
> > > > > > > cleaner
> > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > >
> > > > > > > > > > I am fine with RemotePartitionRemover or
> > > > > RemoteLogDeletionManager(we
> > > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > > >
> > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > processes
> > > > it
> > > > > if
> > > > > > > it is
> > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > >
> > > > > > > > > > This is to handle duplicate delete_partition_marked
> > events. RLC
> > > > > > > > > > internally maintains a state for the delete_partition
> > events
> > > > and
> > > > > if
> > > > > > > it
> > > > > > > > > > already has an existing event then it ignores if it is
> > already
> > > > > being
> > > > > > > > > > processed.
> > > > > > > > > >
> > > > > > > > > > 5117. Should we add a new MessageFormatter to read the
> tier
> > > > > metadata
> > > > > > > > > topic?
> > > > > > > > > >
> > > > > > > > > > Right, this is in plan but did not mention it in the KIP.
> > This
> > > > > will
> > > > > > > be
> > > > > > > > > > useful for debugging purposes too.
> > > > > > > > > >
> > > > > > > > > > 5118. "Maximum remote log reader thread pool task queue
> > size.
> > > > If
> > > > > the
> > > > > > > task
> > > > > > > > > > queue is full, broker will stop reading remote log
> > segments."
> > > > > What
> > > > > > > do we
> > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > >
> > > > > > > > > > We return an error response for that partition.
> > > > > > > > > >
> > > > > > > > > > 5119. It would be useful to list all things not supported
> > in
> > > > the
> > > > > > > first
> > > > > > > > > > version in a Future work or Limitations section. For
> > example,
> > > > > > > compacted
> > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from true
> > to
> > > > > false,
> > > > > > > etc.
> > > > > > > > > >
> > > > > > > > > > We already have a non-goals section which is filled with
> > some
> > > > of
> > > > > > > these
> > > > > > > > > > details. Do we need another limitations section?
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Satish.
> > > > > > > > > >
> > > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <
> jun@confluent.io>
> > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > Hi, Satish,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > > > >
> > > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting
> the
> > > > leader
> > > > > > > epoch
> > > > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > > > mentioned
> > > > > > > an
> > > > > > > > > issue
> > > > > > > > > > > earlier. Suppose the leader's local start offset is
> 100.
> > The
> > > > > > > follower
> > > > > > > > > finds
> > > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > producerState
> > > > > > > > > with
> > > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > producerState
> > > > > > > to
> > > > > > > > > > > offset 100 requires more work since one needs to
> > download the
> > > > > > > previous
> > > > > > > > > > > producerState up to offset 80 and then replay the
> > messages
> > > > > from 80
> > > > > > > to
> > > > > > > > > 100.
> > > > > > > > > > > It seems that it's simpler in this case for the
> follower
> > just
> > > > > to
> > > > > > > take
> > > > > > > > > the
> > > > > > > > > > > remote segment as it is and start fetching from offset
> > 120.
> > > > > > > > > > >
> > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems
> that
> > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on
> the
> > > > > replicas
> > > > > > > for
> > > > > > > > > a
> > > > > > > > > > > partition, not on the replicas for the
> > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > > > >
> > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> implemented
> > now.
> > > > > > > Could you
> > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > >
> > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > following
> > > > two
> > > > > > > APIs
> > > > > > > > > are
> > > > > > > > > > > used. According to the wiki, the former is used for
> topic
> > > > > deletion
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > latter is used for retention. It seems that retention
> > should
> > > > > use
> > > > > > > the
> > > > > > > > > former
> > > > > > > > > > > since remote segments without a matching epoch in the
> > leader
> > > > > > > > > (potentially
> > > > > > > > > > > due to unclean leader election) also need to be garbage
> > > > > collected.
> > > > > > > The
> > > > > > > > > > > latter seems to be used for the new leader to determine
> > the
> > > > > last
> > > > > > > tiered
> > > > > > > > > > > segment.
> > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > >
> > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems
> > that
> > > > > they
> > > > > > > can
> > > > > > > > > > > use RemoteLogSegmentId instead of
> > RemoteLogSegmentMetadata.
> > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > instead
> > > > of
> > > > > > > Long?
> > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > implementation and
> > > > > > > > > others
> > > > > > > > > > > don't?
> > > > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> pass
> > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than
> a
> > file
> > > > > > > since it
> > > > > > > > > will
> > > > > > > > > > > be generated in memory.
> > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs
> > both
> > > > > > > > > baseOffset and
> > > > > > > > > > > startOffset. For example, deleteRecords() could move
> the
> > > > > > > startOffset
> > > > > > > > > to the
> > > > > > > > > > > middle of a segment. If we copy the full segment to
> > remote
> > > > > > > storage, the
> > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > is
> > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > >
> > > > > > > > > > > 5103. configs:
> > > > > > > > > > > 5103.1 Could we define the default value of
> non-required
> > > > > configs
> > > > > > > (e.g
> > > > > > > > > the
> > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > default
> > > > to
> > > > > > > > > retention.ms,
> > > > > > > > > > > instead of remote.log.retention.minutes. Similarly, it
> > seems
> > > > > > > > > > > that local.log.retention.bytes should default to
> > > > segment.bytes.
> > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > description
> > > > > says
> > > > > > > "used
> > > > > > > > > in
> > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > indexes
> > > > and
> > > > > > > clean
> > > > > > > > > up
> > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > config remote.log.reader.threads for fetching remote
> > data.
> > > > It's
> > > > > > > weird
> > > > > > > > > to
> > > > > > > > > > > fetch remote index and log in different thread pools
> > since
> > > > > both are
> > > > > > > > > used
> > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that
> the
> > > > > amount of
> > > > > > > > > time to
> > > > > > > > > > > back off when there is no work to do? If so, perhaps it
> > can
> > > > be
> > > > > > > renamed
> > > > > > > > > as
> > > > > > > > > > > backoff.ms.
> > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > rlm_retry_interval_ms
> > > > > > > configs?
> > > > > > > > > If
> > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > >
> > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of
> the
> > > > index
> > > > > > > files
> > > > > > > > > of
> > > > > > > > > > > remote log segments to avoid multiple index fetches
> from
> > the
> > > > > remote
> > > > > > > > > > > storage." Is the RLM in memory or on disk? If on disk,
> > where
> > > > > is it
> > > > > > > > > stored?
> > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > >
> > > > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest
> > Local
> > > > > > > Offset in
> > > > > > > > > > > different places. It would be useful to standardize the
> > > > > > > terminology.
> > > > > > > > > > >
> > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state".
> It
> > > > > listed two
> > > > > > > > > options
> > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > >
> > > > > > > > > > > 5107. Follower to leader transition: It has step 2, but
> > not
> > > > > step 1.
> > > > > > > > > > >
> > > > > > > > > > > 5108. If a consumer fetches from the remote data and
> the
> > > > remote
> > > > > > > > > storage is
> > > > > > > > > > > not available, what error code is used in the fetch
> > response?
> > > > > > > > > > >
> > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the
> > first
> > > > > > > message
> > > > > > > > > > > offset whose timestamp is >= to the given timestamp in
> > the
> > > > > request.
> > > > > > > > > That
> > > > > > > > > > > means it checks in remote log time indexes first, after
> > which
> > > > > > > local log
> > > > > > > > > > > time indexes are checked." Could you document which
> > method in
> > > > > RLMM
> > > > > > > is
> > > > > > > > > used
> > > > > > > > > > > for this?
> > > > > > > > > > >
> > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > > > > metadata of
> > > > > > > that
> > > > > > > > > > > partition with a delete marker and publishes them to
> > RLMM."
> > > > > This
> > > > > > > seems
> > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > >
> > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > > earliest
> > > > > > > leader
> > > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > > int
> > > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> > Do we
> > > > > need
> > > > > > > that
> > > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > > >
> > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > fetching
> > > > > from
> > > > > > > remote
> > > > > > > > > > > storage?
> > > > > > > > > > >
> > > > > > > > > > > 5113. "Committed offsets can be stored in a local file
> to
> > > > avoid
> > > > > > > > > reading the
> > > > > > > > > > > messages again when a broker is restarted." Could you
> > > > describe
> > > > > the
> > > > > > > > > format
> > > > > > > > > > > and the location of the file? Also, could the same
> > message be
> > > > > > > > > processed by
> > > > > > > > > > > RLMM again after broker restart? If so, how do we
> handle
> > > > that?
> > > > > > > > > > >
> > > > > > > > > > > 5114. Message format
> > > > > > > > > > > 5114.1 There are two records named
> > > > > RemoteLogSegmentMetadataRecord
> > > > > > > with
> > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> document
> > > > > whether
> > > > > > > > > endOffset
> > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> explain
> > > > > > > LeaderEpoch a
> > > > > > > > > bit
> > > > > > > > > > > more? Is that the epoch of the leader when it copies
> the
> > > > > segment to
> > > > > > > > > remote
> > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> more?
> > > > Each
> > > > > > > record
> > > > > > > > > in
> > > > > > > > > > > Kafka already has a timestamp field. Could we just use
> > that?
> > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > > >
> > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> with
> > the
> > > > > log
> > > > > > > > > cleaner
> > > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > >
> > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > processes
> > > > > it
> > > > > > > if it
> > > > > > > > > is
> > > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > >
> > > > > > > > > > > 5117. Should we add a new MessageFormatter to read the
> > tier
> > > > > > > metadata
> > > > > > > > > topic?
> > > > > > > > > > >
> > > > > > > > > > > 5118. "Maximum remote log reader thread pool task queue
> > size.
> > > > > If
> > > > > > > the
> > > > > > > > > task
> > > > > > > > > > > queue is full, broker will stop reading remote log
> > segments."
> > > > > What
> > > > > > > do
> > > > > > > > > we
> > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > >
> > > > > > > > > > > 5119. It would be useful to list all things not
> > supported in
> > > > > the
> > > > > > > first
> > > > > > > > > > > version in a Future work or Limitations section. For
> > example,
> > > > > > > compacted
> > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> > true to
> > > > > false,
> > > > > > > > > etc.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > > (
> > > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > >
> > > >
> >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Jun,
Thanks for your comments. Please find the inline replies below.

9300. Could we add a section on downgrade?

Added Downgrade section with the details as discussed.

9301. "Feature test cases and test results are documented in this google
spreadsheet <https://docs.google.com/spreadsheets/d/1XhNJqjzwXvMCcAOhEH0sSXU6RTvyoSf93DHF-YMfGLk/edit?usp=sharing>
." The google spreadsheet doesn't have a link. Could we add the plan
for integration and system systems?

For integration tests, we use file based(LocalTieredStorage)
RemoteStorageManager(RSM) . For system tests, we plan to have a single
node HDFS cluster in one of the containers and use HDFS RSM
implementation. This is updated in the KIP.  Doc for integration and
system testing doc is in progress. We will share it once it is ready.

~Satish.

On Tue, 9 Feb 2021 at 01:34, Jun Rao <ju...@confluent.io> wrote:
>
> Hi, Satish,
>
> Thanks for the reply. Just a couple of more comments.
>
> 9300. Could we add a section on downgrade?
>
> 9301. "Feature test cases and test results are documented in this google
> spreadsheet
> <https://docs.google.com/spreadsheets/d/1XhNJqjzwXvMCcAOhEH0sSXU6RTvyoSf93DHF-YMfGLk/edit?usp=sharing>
> ." The google spreadsheet doesn't have a link. Could we add the plan for
> integration and system systems?
>
> Jun
>
> On Mon, Feb 8, 2021 at 8:21 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi Manikumar,
> > Thanks for your comment. Please find the inline replies below.
> >
> > 9201. Can we make it clear by mentioning RLM as an internal component and
> > RLMM and RSM are pluggable components.
> > It's also good to update/add the RLM, RLMM components  diagrams with
> > internal components/tasks.
> >
> > Updated the KIP.
> >
> > 9202. RLMM metadata topic is a non-compact topic. Can we make clear that
> > metadata topic retention time should be greater
> >  than user topics. or How about setting retention set to -1 (unlimited) and
> > let users to modify it.
> >
> > I am fine with -1 and updated the configuration doc and made it clear
> > in the KIP.
> >
> > 9203. RLMM has additional responsibility of handling topic delete requests.
> > With current design, RLMM implementations need
> > to implement deletion logic. How about making RemotePartitionRemover as a
> > separate task, which can be reused..
> >
> > Good point. We had plans to address that in a future KIP, added in
> > future work items.
> >
> > 9204. Can we list all the new error codes added by the KIP?
> >
> > Those are mentioned in the protocol changes section.
> >
> > 9205. Can we also support  `remote.log.metadata.producr.*`,
> > `remote.log.metadata.consumer.*` prefixed configs to
> > override any default configs ?. also include RLMM cache configs, if any.
> >
> > Good point. Updated the KIP.
> >
> > 9206. In the Upgrade section, we have below statement
> > "Before enabling tiered storage, you should make sure the producer
> > snapshots are built for all the segments for that topic
> > in all followers. You should wait till the log retention occurs for all the
> > segments so that all the segments have producer snapshots."
> > Can you add more details about this constraint?
> >
> > Sure, updated in the KIP.
> >
> >
> > ~Satish.
> >
> > On Tue, 2 Feb 2021 at 21:04, Manikumar <ma...@gmail.com> wrote:
> > >
> > > Hi Satish,
> > >
> > > Thanks for the KIP.  some of my comments below.
> > >
> > > 9201. Can we make it clear by mentioning RLM as an internal component and
> > > RLMM and RSM are pluggable components.
> > > It's also good to update/add the RLM, RLMM components  diagrams with
> > > internal components/tasks.
> > >
> > > 9202. RLMM metadata topic is a non-compact topic. Can we make clear that
> > > metadata topic retention time should be greater
> > >  than user topics. or How about setting retention set to -1 (unlimited)
> > and
> > > let users to modify it.
> > >
> > > 9203. RLMM has additional responsibility of handling topic delete
> > requests.
> > > With current design, RLMM implementations need
> > > to implement deletion logic. How about making RemotePartitionRemover as a
> > > separate task, which can be reused..
> > >
> > > 9204. Can we list all the new error codes added by the KIP?
> > >
> > > 9205. Can we also support  `remote.log.metadata.producr.*`,
> > > `remote.log.metadata.consumer.*` prefixed configs to
> > > override any default configs ?. also include RLMM cache configs, if any.
> > >
> > > 9206. In the Upgrade section, we have below statement
> > > "Before enabling tiered storage, you should make sure the producer
> > > snapshots are built for all the segments for that topic
> > > in all followers. You should wait till the log retention occurs for all
> > the
> > > segments so that all the segments have producer snapshots."
> > >
> > >  Can you add more details about this constraint?
> > >
> > >
> > > Thanks,
> > > Manikumar
> > >
> > > On Tue, Jan 26, 2021 at 1:12 AM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Satish,
> > > >
> > > > Thanks for the reply. A few more comments below.
> > > >
> > > > 6000. When RSM/RLMM is not available, it seems that we need to return a
> > > > retriable error to the affected client requests. So, should we add a
> > new
> > > > error code for requests like fetch and listOffset?
> > > >
> > > > 9111. RemotePartitionRemover:
> > > > 91111.1 Could we make it clear that it's part of the default RLMM
> > > > implementation and a separate implementation is needed if one has a
> > > > customized RLMM?
> > > > 91111.2 "RPRM gets all the remote log segments for the partition using
> > RLMM
> > > > and each of these remote log segments is deleted with the next
> > steps.RLMM
> > > > subscribes to the local remote log metadata partitions and it will
> > have the
> > > > segment metadata of all the user topic partitions associated with that
> > > > remote log metadata partition." It seems that RLMM needs to subscribe
> > to
> > > > the remote log metadata partitions first before those segments can be
> > > > deleted?
> > > > 9111.3 There are still references to "remote log cleaners". They need
> > to be
> > > > replaced with RemotePartitionRemover.
> > > >
> > > > 9114.1 Could we add the requirement on log.message.format before
> > enabling
> > > > tiered storage?
> > > >
> > > > 9116. RemoteLogMetadataFormatter: This is used with the ConsoleConsumer
> > > > tool, right? Then, are those new options for ConsoleConsumer and how
> > are
> > > > they passed in?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Mon, Jan 25, 2021 at 8:02 AM Satish Duggana <
> > satish.duggana@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Jun,
> > > > >
> > > > > Thanks for your comments. Please find the inline replies below.
> > > > > Several of these were discussed/clarified in our last discussion.
> > > > >
> > > > > 6000. Since we are returning new error codes, we need to bump up the
> > > > > protocol version for Fetch request. Also, it will be useful to
> > document
> > > > all
> > > > > new error codes and whether they are retriable or not.
> > > > > This previous comment doesn't seem to be addressed.
> > > > >
> > > > > Updated KIP on fetch protocol and error code.
> > > > >
> > > > > 9110. flat_file_format: Could you define the serialized
> > representation
> > > > for
> > > > > each field in the header?
> > > > >
> > > > > Added the serialized representation types in the KIP.
> > > > >
> > > > > 9111. RPM has the following 2 steps.
> > > > > "1. The controller publishes delete_partition_marked event to say
> > that
> > > > the
> > > > > partition is marked for deletion. There can be multiple events
> > published
> > > > > when the controller restarts or failover and this event will be
> > > > > deduplicated by RPM.
> > > > > 2. RPM receives the delete_partition_marked and processes it if it
> > is not
> > > > > yet processed earlier."
> > > > > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> > > > default
> > > > > RLMM implementation or is it meant for any RLMM implementation?
> > > > >
> > > > > RPM(or RPRM) is only for the default RLMM implementation. RPM
> > receives
> > > > > the delete_partition_marked events from RLMM and acts on them.
> > Updated
> > > > > KIP with more details.
> > > > >
> > > > > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful
> > to
> > > > make
> > > > > it clear in the comment that this is for the initial retry backoff.
> > > > >
> > > > > Updated the KIP.
> > > > >
> > > > > 9113. RLMM:
> > > > > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> > > > updated.
> > > > >
> > > > > Updated javadoc of this method.
> > > > >
> > > > > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch
> > field
> > > > > since all other update events have leaderEpoch?
> > > > >
> > > > > +1 to have that,  updated the KIP.
> > > > >
> > > > > 9113.3 Could we rename RemotePartitionState to
> > RemotePartitionDeleteState
> > > > > to make it clear this is for deletion?
> > > > >
> > > > > Sure, updated the KIP.
> > > > >
> > > > > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to
> > make it
> > > > > clear this is for segment?
> > > > >
> > > > > Sure, Updated the KIP.
> > > > >
> > > > > 9114.Upgrade:
> > > > > 9114.1 It seems that we require message format > 0.11 for turning on
> > the
> > > > > remote store feature.
> > > > >
> > > > > 9114.2 It's not clear to me why remote.log.storage.system.enable
> > needs to
> > > > > be set to true before bumping up inter.broker.protocol.version.
> > > > >
> > > > > Agree that based on our discussion, this is not required. Upgrade
> > > > > notes is updated.
> > > > >
> > > > > 9114.3 "If the topic-id is not received in the LeaderAndIsr request
> > then
> > > > > remote log storage will not start. But it will log an error message
> > in
> > > > the
> > > > > log. One way to address this is to do a rolling restart of that
> > broker,
> > > > so
> > > > > that the leader will be moved to another broker and the controller
> > will
> > > > > send LeaderAndIsr with the registered topic-id." Why is this needed?
> > With
> > > > > KIP-516, after upgrading to the latest protocol, topicIds are auto
> > > > created.
> > > > >
> > > > > I thought there may be edge cases of not receiving topic-id but you
> > > > > clarified in the meeting that won’t be the case. I agree that it is
> > > > > not needed.
> > > > >
> > > > > 9115. testing: Could you comment on how we plan to do integration and
> > > > > system tests? Do we plan to include, for example, an in-memory
> > > > > implementation of RSM?
> > > > >
> > > > > For integration tests, we can have file based RSM that we have. For
> > > > > system tests, we can have a single node HDFS cluster in one of the
> > > > > containers and use HDFS RSM implementation.
> > > > >
> > > > > 9116. There is still a todo for the message formatter.
> > > > >
> > > > > Updated the KIP with the format.
> > > > >
> > > > >
> > > > > Satish.
> > > > >
> > > > >
> > > > > On Tue, 12 Jan 2021 at 07:07, Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > Hi, Satish,
> > > > > >
> > > > > > Thanks for the reply. A few more followup comments.
> > > > > >
> > > > > > 6000. Since we are returning new error codes, we need to bump up
> > the
> > > > > > protocol version for Fetch request. Also, it will be useful to
> > document
> > > > > all
> > > > > > new error codes and whether they are retriable or not.
> > > > > > This previous comment doesn't seem to be addressed.
> > > > > >
> > > > > > 9110. flat_file_format: Could you define the serialized
> > representation
> > > > > for
> > > > > > each field in the header?
> > > > > >
> > > > > > 9111. RPM has the following 2 steps.
> > > > > > "1. The controller publishes delete_partition_marked event to say
> > that
> > > > > the
> > > > > > partition is marked for deletion. There can be multiple events
> > > > published
> > > > > > when the controller restarts or failover and this event will be
> > > > > > deduplicated by RPM.
> > > > > > 2. RPM receives the delete_partition_marked and processes it if it
> > is
> > > > not
> > > > > > yet processed earlier."
> > > > > > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> > > > > default
> > > > > > RLMM implementation or is it meant for any RLMM implementation?
> > > > > >
> > > > > > 9112. remote.log.manager.task.retry.backoff.ms: It would be
> > useful to
> > > > > make
> > > > > > it clear in the comment that this is for the initial retry backoff.
> > > > > >
> > > > > > 9113. RLMM:
> > > > > > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> > > > > updated.
> > > > > > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch
> > field
> > > > > > since all other update events have leaderEpoch?
> > > > > > 9113.3 Could we rename RemotePartitionState to
> > > > RemotePartitionDeleteState
> > > > > > to make it clear this is for deletion?
> > > > > > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to
> > make
> > > > it
> > > > > > clear this is for segment?
> > > > > >
> > > > > > 9114.Upgrade:
> > > > > > 9114.1 It seems that we require message format > 0.11 for turning
> > on
> > > > the
> > > > > > remote store feature.
> > > > > > 9114.2 It's not clear to me why remote.log.storage.system.enable
> > needs
> > > > to
> > > > > > be set to true before bumping up inter.broker.protocol.version.
> > > > > > 9114.3 "If the topic-id is not received in the LeaderAndIsr request
> > > > then
> > > > > > remote log storage will not start. But it will log an error
> > message in
> > > > > the
> > > > > > log. One way to address this is to do a rolling restart of that
> > broker,
> > > > > so
> > > > > > that the leader will be moved to another broker and the controller
> > will
> > > > > > send LeaderAndIsr with the registered topic-id." Why is this
> > needed?
> > > > With
> > > > > > KIP-516, after upgrading to the latest protocol, topicIds are auto
> > > > > created.
> > > > > >
> > > > > > 9115. testing: Could you comment on how we plan to do integration
> > and
> > > > > > system tests? Do we plan to include, for example, an in-memory
> > > > > > implementation of RSM?
> > > > > >
> > > > > > 9116. There is still a todo for the message formatter.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Jun,
> > > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > > >
> > > > > > > 6022. For packages used for server plugins, the convention is to
> > > > > > > use org.apache.kafka.server. See java-based Authorizer as an
> > example.
> > > > > > >
> > > > > > > Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> > > > > > > ‘org.apache.kafka.server.log.remote.storage’.  Updated in the
> > KIP.
> > > > > > >
> > > > > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format?
> > The
> > > > > flat
> > > > > > > file captures the state of the remote segments. After a
> > partition is
> > > > > > > deleted, it seems that we just need to remove the partitions's
> > remote
> > > > > > > segments from the flat file.
> > > > > > >
> > > > > > > DeletePartitionState might not yet have been processed by RPM
> > and not
> > > > > > > completed. We  will not have  that in flat  file format  once it
> > > > > > > reaches DELETE_PARTITION_FINISHED state.
> > > > > > >
> > > > > > > 9101. Upgrade: It will be useful to allow direct upgrade from an
> > old
> > > > > > > version. It seems that's doable. One can just do the normal
> > upgrade
> > > > > first
> > > > > > > and wait enough time (for producer snapshots to be built), and
> > then
> > > > > enable
> > > > > > > remote storage.
> > > > > > >
> > > > > > > Upgrade notes updates in the KIP.
> > > > > > >
> > > > > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM
> > starts
> > > > > > > tracking the remote segments when
> > RLMM.onPartitionLeadershipChanges()
> > > > > is
> > > > > > > called with the broker being the leader for __remote_log_metadata
> > > > > > > partition? If so, could we document it?
> > > > > > >
> > > > > > > The current plan is to have that as part of RLMM and RPM uses
> > that to
> > > > > > > get the remote segments list. I will add this detail in the
> > > > respective
> > > > > > > sections.
> > > > > > >
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> > > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the reply. A few more followup comments.
> > > > > > > >
> > > > > > > > 6022. For packages used for server plugins, the convention is
> > to
> > > > > > > > use org.apache.kafka.server. See java-based Authorizer as an
> > > > example.
> > > > > > > >
> > > > > > > > 9100. Do we need DeletePartitionStateRecord in
> > flat_file_format?
> > > > The
> > > > > flat
> > > > > > > > file captures the state of the remote segments. After a
> > partition
> > > > is
> > > > > > > > deleted, it seems that we just need to remove the partitions's
> > > > remote
> > > > > > > > segments from the flat file.
> > > > > > > >
> > > > > > > > 9101. Upgrade: It will be useful to allow direct upgrade from
> > an
> > > > old
> > > > > > > > version. It seems that's doable. One can just do the normal
> > upgrade
> > > > > first
> > > > > > > > and wait enough time (for producer snapshots to be built), and
> > then
> > > > > > > enable
> > > > > > > > remote storage.
> > > > > > > >
> > > > > > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM
> > > > starts
> > > > > > > > tracking the remote segments when
> > > > > RLMM.onPartitionLeadershipChanges() is
> > > > > > > > called with the broker being the leader for
> > __remote_log_metadata
> > > > > > > > partition? If so, could we document it?
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <
> > > > > kprakasam@confluent.io
> > > > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Satish,
> > > > > > > > >
> > > > > > > > > Thanks for the updates! A few more comments below.
> > > > > > > > >
> > > > > > > > > 9001. Under the "Upgrade" section, there is a line
> > mentioning:
> > > > > > > "Upgrade the
> > > > > > > > > existing Kafka cluster to 2.7 version and allow this to run
> > for
> > > > > the log
> > > > > > > > > retention of user topics that you want to enable tiered
> > storage.
> > > > > This
> > > > > > > will
> > > > > > > > > allow all the topics to have the producer snapshots
> > generated for
> > > > > each
> > > > > > > log
> > > > > > > > > segment." -- Which associated change in AK were you
> > referring to
> > > > > here?
> > > > > > > Is
> > > > > > > > > it: https://github.com/apache/kafka/pull/7929 ? It seems
> > like I
> > > > > don't
> > > > > > > see
> > > > > > > > > it in the 2.7 release branch yet, here is the link:
> > > > > > > > > https://github.com/apache/kafka/commits/2.7.
> > > > > > > > >
> > > > > > > > > 9002. Under the "Upgrade" section, the configuration
> > mentioned is
> > > > > > > > > 'remote.log.storage.system.enable'. However, under "Public
> > > > > Interfaces"
> > > > > > > > > section the corresponding configuration is
> > > > > > > 'remote.storage.system.enable'.
> > > > > > > > > Could we use the same one in both, maybe
> > > > > > > > > 'remote.log.storage.system.enable'?
> > > > > > > > >
> > > > > > > > > 9003. Under "Per Topic Configuration", the KIP recommends
> > setting
> > > > > > > > > 'remote.log.storage.enable' to true at a per-topic level. It
> > will
> > > > > be
> > > > > > > useful
> > > > > > > > > to add a line that if the user wants to enable it for all
> > topics,
> > > > > then
> > > > > > > they
> > > > > > > > > should be able to set the cluster-wide default to true.
> > Also, it
> > > > > will
> > > > > > > be
> > > > > > > > > useful to mention that the KIP currently does not support
> > setting
> > > > > it to
> > > > > > > > > false (after it is set to true), and add that to the future
> > work
> > > > > > > section.
> > > > > > > > >
> > > > > > > > > 9004. Under "Committed offsets file format", the sample
> > provided
> > > > > shows
> > > > > > > > > partition number and offset. Is the topic name required for
> > > > > identifying
> > > > > > > > > which topic the partitions belong to?
> > > > > > > > >
> > > > > > > > > 9005. Under "Internal flat-file store format of remote log
> > > > > metadata",
> > > > > > > it
> > > > > > > > > seems useful to specify both topic name and topic ID for
> > > > debugging
> > > > > > > > > purposes.
> > > > > > > > >
> > > > > > > > > 9006. Under "Internal flat-file store format of remote log
> > > > > metadata",
> > > > > > > the
> > > > > > > > > description of "metadata-topic-offset" currently says
> > "offset of
> > > > > the
> > > > > > > remote
> > > > > > > > > log metadata topic from which this topic partition's remote
> > log
> > > > > > > metadata is
> > > > > > > > > fetched." Just for the wording, perhaps you meant to refer
> > to the
> > > > > > > offset
> > > > > > > > > upto which the file has been committed? i.e. "offset of the
> > > > remote
> > > > > log
> > > > > > > > > metadata topic upto which this topic partition's remote log
> > > > > metadata
> > > > > > > has
> > > > > > > > > been committed into this file."
> > > > > > > > >
> > > > > > > > > 9007. Under "Internal flat-file store format of remote log
> > > > > metadata",
> > > > > > > the
> > > > > > > > > schema of the payload (i.e. beyond the header) seems to
> > contain
> > > > the
> > > > > > > events
> > > > > > > > > from the metadata topic. It seems useful to instead persist
> > the
> > > > > > > > > representation of the materialized state of the events, so
> > that
> > > > > for the
> > > > > > > > > same segment only the latest state is stored. Besides
> > reducing
> > > > > storage
> > > > > > > > > footprint, this also is likely to relate directly with the
> > > > > in-memory
> > > > > > > > > representation of the RLMM cache (which probably is some
> > kind of
> > > > a
> > > > > Map
> > > > > > > with
> > > > > > > > > key being segment ID and value being the segment state), so
> > > > > recovery
> > > > > > > from
> > > > > > > > > disk will be straightforward.
> > > > > > > > >
> > > > > > > > > 9008. Under "Topic deletion lifecycle", step (1), it will be
> > > > > useful to
> > > > > > > > > mention when in the deletion flow does the controller
> > publish the
> > > > > > > > > delete_partition_marked event to say that the partition is
> > marked
> > > > > for
> > > > > > > > > deletion?
> > > > > > > > >
> > > > > > > > > 9009. There are ~4 TODOs in the KIP. Could you please address
> > > > > these or
> > > > > > > > > remove them?
> > > > > > > > >
> > > > > > > > > 9010. There is a reference to a Google doc on the KIP which
> > was
> > > > > used
> > > > > > > > > earlier for discussions. Please could you remove the
> > reference,
> > > > > since
> > > > > > > the
> > > > > > > > > KIP is the source of the truth?
> > > > > > > > >
> > > > > > > > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Cheers,
> > > > > > > > > Kowshik
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> > > > > > > satish.duggana@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Jun,
> > > > > > > > > > Thanks for your comments. Please go through the inline
> > replies.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 5102.2: It seems that both positions can just be int.
> > Another
> > > > > option
> > > > > > > is
> > > > > > > > > to
> > > > > > > > > > have two methods. Would it be clearer?
> > > > > > > > > >
> > > > > > > > > >     InputStream
> > fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > remoteLogSegmentMetadata,  int startPosition)
> > > > > > > > > throwsRemoteStorageException;
> > > > > > > > > >
> > > > > > > > > >     InputStream
> > fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > remoteLogSegmentMetadata, int startPosition, int
> > endPosition)
> > > > > throws
> > > > > > > > > > RemoteStorageException;
> > > > > > > > > >
> > > > > > > > > > That makes sense to me, updated the KIP.
> > > > > > > > > >
> > > > > > > > > > 6003: Could you also update the javadoc for the return
> > value?
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > > > log.retention.bytes
> > > > > > > > > > to be consistent with local.log.retention.ms?
> > > > > > > > > >
> > > > > > > > > > Yes, it can be defaulted to log.retention.bytes.
> > > > > > > > > >
> > > > > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > > > > >
> > > > > > > > > > Added TopicIdPartition in the KIP.
> > > > > > > > > >
> > > > > > > > > > 6022: For all public facing classes, could you specify the
> > > > > package
> > > > > > > name?
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Satish.
> > > > > > > > > >
> > > > > > > > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io>
> > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > Hi, Satish,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the reply. A few more comments below.
> > > > > > > > > > >
> > > > > > > > > > > 5102.2: It seems that both positions can just be int.
> > Another
> > > > > > > option is
> > > > > > > > > > to
> > > > > > > > > > > have two methods. Would it be clearer?
> > > > > > > > > > >
> > > > > > > > > > >     InputStream
> > fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > remoteLogSegmentMetadata,
> > > > > > > > > > >                                     int startPosition)
> > throws
> > > > > > > > > > > RemoteStorageException;
> > > > > > > > > > >
> > > > > > > > > > >     InputStream
> > fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > remoteLogSegmentMetadata,
> > > > > > > > > > >                                     int startPosition,
> > int
> > > > > > > endPosition)
> > > > > > > > > > > throws RemoteStorageException;
> > > > > > > > > > >
> > > > > > > > > > > 6003: Could you also update the javadoc for the return
> > value?
> > > > > > > > > > >
> > > > > > > > > > > 6010: What kind of tiering throughput have you seen with
> > 5
> > > > > threads?
> > > > > > > > > > >
> > > > > > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > > > > > log.retention.bytes
> > > > > > > > > > > to be consistent with local.log.retention.ms?
> > > > > > > > > > >
> > > > > > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > > > > > >
> > > > > > > > > > > 6022: For all public facing classes, could you specify
> > the
> > > > > package
> > > > > > > > > name?
> > > > > > > > > > >
> > > > > > > > > > > It seems that you already added the topicId support. Two
> > > > other
> > > > > > > > > remaining
> > > > > > > > > > > items are (a) the format of local tier metadata storage
> > and
> > > > (b)
> > > > > > > > > upgrade.
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > Thanks for your comments. Please find the inline
> > replies
> > > > > below.
> > > > > > > > > > > >
> > > > > > > > > > > > >605.2 It's rare for the follower to need the remote
> > data.
> > > > > So,
> > > > > > > the
> > > > > > > > > > current
> > > > > > > > > > > > approach is fine too. Could you document the process of
> > > > > > > rebuilding
> > > > > > > > > the
> > > > > > > > > > > > producer state since we can't simply trim the
> > producerState
> > > > > to an
> > > > > > > > > > offset in
> > > > > > > > > > > > the middle of a segment.
> > > > > > > > > > > >
> > > > > > > > > > > > Will clarify in the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > >5102.2 Would it be clearer to make startPosiont long
> > and
> > > > > > > endPosition
> > > > > > > > > > of
> > > > > > > > > > > > Optional<Long>?
> > > > > > > > > > > >
> > > > > > > > > > > > We will have arg checks with respective validation. It
> > is
> > > > > not a
> > > > > > > good
> > > > > > > > > > > > practice to have arguments with optional as mentioned
> > here.
> > > > > > > > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as
> > File
> > > > > > > instead of
> > > > > > > > > > > > ByteBuffer.
> > > > > > > > > > > >
> > > > > > > > > > > > Updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >5102.7 Could you define all public methods for
> > > > > LogSegmentData?
> > > > > > > > > > > >
> > > > > > > > > > > > Updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >5103.5 Could you change the reference to
> > > > > > > rlm_process_interval_ms and
> > > > > > > > > > > > rlm_retry_interval_ms to the new config names? Also,
> > the
> > > > > retry
> > > > > > > > > interval
> > > > > > > > > > > > config seems still missing. It would be useful to
> > support
> > > > > > > exponential
> > > > > > > > > > > > backoff with the retry interval config.
> > > > > > > > > > > >
> > > > > > > > > > > > Good point. We wanted the retry with truncated
> > exponential
> > > > > > > backoff,
> > > > > > > > > > > > updated the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > >5111. "RLM follower fetches the earliest offset for
> > the
> > > > > earliest
> > > > > > > > > > leader
> > > > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > topicPartition,
> > > > > > > > > > int
> > > > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> > > > This
> > > > > > > text is
> > > > > > > > > > still
> > > > > > > > > > > > there. Also, could we remove earliestLogOffset() from
> > RLMM?
> > > > > > > > > > > >
> > > > > > > > > > > > Updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >5115. There are still references to "remote log
> > cleaners".
> > > > > > > > > > > >
> > > > > > > > > > > > Updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >6000. Since we are returning new error codes, we need
> > to
> > > > > bump
> > > > > > > up the
> > > > > > > > > > > > protocol version for Fetch request. Also, it will be
> > useful
> > > > > to
> > > > > > > > > > document all
> > > > > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, we will add that in the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs():
> > > > > Currently,
> > > > > > > > > > leaderEpoch
> > > > > > > > > > > > is int32 instead of long.
> > > > > > > > > > > >
> > > > > > > > > > > > Updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> > > > needed
> > > > > > > given
> > > > > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > > > > >
> > > > > > > > > > > > No, it is fixed.
> > > > > > > > > > > >
> > > > > > > > > > > > >6003. RemoteLogSegmentMetadata
> > > > > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > > > topicPartition, long offset, int epochForOffset):
> > Should
> > > > this
> > > > > > > return
> > > > > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > > > > >
> > > > > > > > > > > > That makes sense, updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >6005. RemoteLogState: It seems it's better to split it
> > > > > between
> > > > > > > > > > > > DeletePartitionUpdate and
> > RemoteLogSegmentMetadataUpdate
> > > > > since
> > > > > > > the
> > > > > > > > > > states
> > > > > > > > > > > > are never shared between the two use cases.
> > > > > > > > > > > >
> > > > > > > > > > > > Agree with that, updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may
> > be ok.
> > > > > > > However,
> > > > > > > > > > is it
> > > > > > > > > > > > ture that other than the metadata topic, RLMM just
> > needs to
> > > > > know
> > > > > > > > > > whether
> > > > > > > > > > > > there is a replica assigned to this broker and doesn't
> > need
> > > > > to
> > > > > > > know
> > > > > > > > > > whether
> > > > > > > > > > > > the replica is the leader or the follower?
> > > > > > > > > > > >
> > > > > > > > > > > > That may be true. If the implementation does not need
> > that,
> > > > > it
> > > > > > > can
> > > > > > > > > > > > ignore the information in the callback.
> > > > > > > > > > > >
> > > > > > > > > > > > >6007: "Handle expired remote segments (leader and
> > > > > follower)":
> > > > > > > Why is
> > > > > > > > > > this
> > > > > > > > > > > > needed in both the leader and the follower?
> > > > > > > > > > > >
> > > > > > > > > > > > Updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > > > > > > > >                 "type": "int64",
> > > > > > > > > > > > The segment size can just be int32.
> > > > > > > > > > > >
> > > > > > > > > > > > Updated.
> > > > > > > > > > > >
> > > > > > > > > > > > >6009. For the record format in the log, it seems that
> > we
> > > > > need
> > > > > > > to add
> > > > > > > > > > > > record
> > > > > > > > > > > > type and record version before the serialized bytes.
> > We can
> > > > > > > follow
> > > > > > > > > the
> > > > > > > > > > > > convention used in
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > > > >
> > > > > > > > > > > > Yes, KIP already mentions that these are serialized
> > before
> > > > > the
> > > > > > > > > payload
> > > > > > > > > > > > as below. We will mention explicitly that these two are
> > > > > written
> > > > > > > > > before
> > > > > > > > > > > > the data is written.
> > > > > > > > > > > >
> > > > > > > > > > > > RLMM instance on broker publishes the message to the
> > topic
> > > > > with
> > > > > > > key
> > > > > > > > > as
> > > > > > > > > > > > null and value with the below format.
> > > > > > > > > > > >
> > > > > > > > > > > > type      : unsigned var int, represents the value
> > type.
> > > > This
> > > > > > > value
> > > > > > > > > is
> > > > > > > > > > > > 'apikey' as mentioned in the schema.
> > > > > > > > > > > > version : unsigned var int, the 'version' number of the
> > > > type
> > > > > as
> > > > > > > > > > > > mentioned in the schema.
> > > > > > > > > > > > data      : record payload in kafka protocol message
> > > > format.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > >6010. remote.log.manager.thread.pool.size: The default
> > > > > value is
> > > > > > > 10.
> > > > > > > > > > This
> > > > > > > > > > > > might be too high when enabling the tiered feature for
> > the
> > > > > first
> > > > > > > > > time.
> > > > > > > > > > > > Since there are lots of segments that need to be tiered
> > > > > > > initially, a
> > > > > > > > > > large
> > > > > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > > > > >
> > > > > > > > > > > > Is the default value 5 reasonable?
> > > > > > > > > > > >
> > > > > > > > > > > > 6011. "The number of milli seconds to keep the local
> > log
> > > > > segment
> > > > > > > > > > before it
> > > > > > > > > > > > gets deleted. If not set, the value in
> > > > > `log.retention.minutes` is
> > > > > > > > > > used. If
> > > > > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > > > log.retention.ms
> > > > > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > > > > Nice typo catch. Updated the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Satish.
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <
> > jun@confluent.io>
> > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > below.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 605.2 It's rare for the follower to need the remote
> > data.
> > > > > So,
> > > > > > > the
> > > > > > > > > > current
> > > > > > > > > > > > > approach is fine too. Could you document the process
> > of
> > > > > > > rebuilding
> > > > > > > > > > the
> > > > > > > > > > > > > producer state since we can't simply trim the
> > > > > producerState to
> > > > > > > an
> > > > > > > > > > offset
> > > > > > > > > > > > in
> > > > > > > > > > > > > the middle of a segment.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.2 Would it be clearer to make startPosiont long
> > and
> > > > > > > > > endPosition
> > > > > > > > > > of
> > > > > > > > > > > > > Optional<Long>?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as
> > File
> > > > > > > instead of
> > > > > > > > > > > > > ByteBuffer.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.7 Could you define all public methods for
> > > > > LogSegmentData?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5103.5 Could you change the reference to
> > > > > > > rlm_process_interval_ms
> > > > > > > > > and
> > > > > > > > > > > > > rlm_retry_interval_ms to the new config names? Also,
> > the
> > > > > retry
> > > > > > > > > > interval
> > > > > > > > > > > > > config seems still missing. It would be useful to
> > support
> > > > > > > > > exponential
> > > > > > > > > > > > > backoff with the retry interval config.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for
> > the
> > > > > > > earliest
> > > > > > > > > > leader
> > > > > > > > > > > > > epoch by calling
> > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > > topicPartition,
> > > > > > > > > > > > int
> > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > offset."
> > > > > This
> > > > > > > text
> > > > > > > > > is
> > > > > > > > > > > > still
> > > > > > > > > > > > > there. Also, could we remove earliestLogOffset() from
> > > > RLMM?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5115. There are still references to "remote log
> > > > cleaners".
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6000. Since we are returning new error codes, we
> > need to
> > > > > bump
> > > > > > > up
> > > > > > > > > the
> > > > > > > > > > > > > protocol version for Fetch request. Also, it will be
> > > > > useful to
> > > > > > > > > > document
> > > > > > > > > > > > all
> > > > > > > > > > > > > new error codes and whether they are retriable or
> > not.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs():
> > > > > Currently,
> > > > > > > > > > > > leaderEpoch
> > > > > > > > > > > > > is int32 instead of long.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> > > > > needed
> > > > > > > given
> > > > > > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6003. RemoteLogSegmentMetadata
> > > > > > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > > > > topicPartition, long offset, int epochForOffset):
> > Should
> > > > > this
> > > > > > > > > return
> > > > > > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be
> > useful
> > > > to
> > > > > > > pick a
> > > > > > > > > > more
> > > > > > > > > > > > > indicative name so that people understand what epoch
> > this
> > > > > is.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6005. RemoteLogState: It seems it's better to split
> > it
> > > > > between
> > > > > > > > > > > > > DeletePartitionUpdate and
> > RemoteLogSegmentMetadataUpdate
> > > > > since
> > > > > > > the
> > > > > > > > > > states
> > > > > > > > > > > > > are never shared between the two use cases.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may
> > be
> > > > ok.
> > > > > > > However,
> > > > > > > > > > is it
> > > > > > > > > > > > > ture that other than the metadata topic, RLMM just
> > needs
> > > > to
> > > > > > > know
> > > > > > > > > > whether
> > > > > > > > > > > > > there is a replica assigned to this broker and
> > doesn't
> > > > > need to
> > > > > > > know
> > > > > > > > > > > > whether
> > > > > > > > > > > > > the replica is the leader or the follower?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6007: "Handle expired remote segments (leader and
> > > > > follower)":
> > > > > > > Why
> > > > > > > > > is
> > > > > > > > > > this
> > > > > > > > > > > > > needed in both the leader and the follower?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > > > > > > > >                 "type": "int64",
> > > > > > > > > > > > > The segment size can just be int32.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6009. For the record format in the log, it seems
> > that we
> > > > > need
> > > > > > > to
> > > > > > > > > add
> > > > > > > > > > > > record
> > > > > > > > > > > > > type and record version before the serialized bytes.
> > We
> > > > can
> > > > > > > follow
> > > > > > > > > > the
> > > > > > > > > > > > > convention used in
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > > > > > .
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6010. remote.log.manager.thread.pool.size: The
> > default
> > > > > value
> > > > > > > is 10.
> > > > > > > > > > This
> > > > > > > > > > > > > might be too high when enabling the tiered feature
> > for
> > > > the
> > > > > > > first
> > > > > > > > > > time.
> > > > > > > > > > > > > Since there are lots of segments that need to be
> > tiered
> > > > > > > initially,
> > > > > > > > > a
> > > > > > > > > > > > large
> > > > > > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6011. "The number of milli seconds to keep the local
> > log
> > > > > > > segment
> > > > > > > > > > before
> > > > > > > > > > > > it
> > > > > > > > > > > > > gets deleted. If not set, the value in
> > > > > `log.retention.minutes`
> > > > > > > is
> > > > > > > > > > used.
> > > > > > > > > > > > If
> > > > > > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > > > > > log.retention.ms
> > > > > > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi,
> > > > > > > > > > > > > > We updated the KIP with the points mentioned in the
> > > > > earlier
> > > > > > > mail
> > > > > > > > > > > > > > except for KIP-516 related changes. You can go
> > through
> > > > > them
> > > > > > > and
> > > > > > > > > > let us
> > > > > > > > > > > > > > know if you have any comments. We will update the
> > KIP
> > > > > with
> > > > > > > the
> > > > > > > > > > > > > > remaining todo items and KIP-516 related changes
> > by end
> > > > > of
> > > > > > > this
> > > > > > > > > > > > > > week(5th Dec).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > Thanks for your comments. Please find the inline
> > > > > replies
> > > > > > > below.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 605.2 "Build the local leader epoch cache by
> > cutting
> > > > > the
> > > > > > > leader
> > > > > > > > > > epoch
> > > > > > > > > > > > > > > sequence received from remote storage to [LSO,
> > > > ELO]." I
> > > > > > > > > > mentioned an
> > > > > > > > > > > > > > issue
> > > > > > > > > > > > > > > earlier. Suppose the leader's local start offset
> > is
> > > > > 100.
> > > > > > > The
> > > > > > > > > > follower
> > > > > > > > > > > > > > finds
> > > > > > > > > > > > > > > a remote segment covering offset range [80,
> > 120). The
> > > > > > > > > > producerState
> > > > > > > > > > > > with
> > > > > > > > > > > > > > > this remote segment is up to offset 120. To trim
> > the
> > > > > > > > > > producerState to
> > > > > > > > > > > > > > > offset 100 requires more work since one needs to
> > > > > download
> > > > > > > the
> > > > > > > > > > > > previous
> > > > > > > > > > > > > > > producerState up to offset 80 and then replay the
> > > > > messages
> > > > > > > from
> > > > > > > > > > 80 to
> > > > > > > > > > > > > > 100.
> > > > > > > > > > > > > > > It seems that it's simpler in this case for the
> > > > > follower
> > > > > > > just
> > > > > > > > > to
> > > > > > > > > > > > take the
> > > > > > > > > > > > > > > remote segment as it is and start fetching from
> > > > offset
> > > > > 120.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We chose that approach to avoid any edge cases
> > here.
> > > > It
> > > > > > > may be
> > > > > > > > > > > > > > > possible that the remote log segment that is
> > received
> > > > > may
> > > > > > > not
> > > > > > > > > > have
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > same leader epoch sequence from 100-120 as it
> > > > contains
> > > > > on
> > > > > > > the
> > > > > > > > > > > > > > > leader(this can happen due to unclean leader).
> > It is
> > > > > safe
> > > > > > > to
> > > > > > > > > > start
> > > > > > > > > > > > > > > from what the leader returns here.Another way is
> > to
> > > > > find
> > > > > > > the
> > > > > > > > > > remote
> > > > > > > > > > > > > > > log segment
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It
> > seems
> > > > > that
> > > > > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only
> > called on
> > > > > the
> > > > > > > > > > replicas
> > > > > > > > > > > > for a
> > > > > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > > obtains the metadata for remote segments for
> > > > deletion.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > RLMM will always receive the callback for the
> > remote
> > > > > log
> > > > > > > > > metadata
> > > > > > > > > > > > > > > topic partitions hosted on the local broker and
> > these
> > > > > will
> > > > > > > be
> > > > > > > > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > > > > implemented
> > > > > > > now.
> > > > > > > > > > Could
> > > > > > > > > > > > you
> > > > > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We mentioned KIP-516 and how it helps. We will
> > update
> > > > > this
> > > > > > > KIP
> > > > > > > > > > with
> > > > > > > > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > > > > > following two
> > > > > > > > > > APIs
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > used. According to the wiki, the former is used
> > for
> > > > > topic
> > > > > > > > > > deletion
> > > > > > > > > > > > and
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > latter is used for retention. It seems that
> > retention
> > > > > > > should
> > > > > > > > > use
> > > > > > > > > > the
> > > > > > > > > > > > > > former
> > > > > > > > > > > > > > > since remote segments without a matching epoch
> > in the
> > > > > > > leader
> > > > > > > > > > > > (potentially
> > > > > > > > > > > > > > > due to unclean leader election) also need to be
> > > > garbage
> > > > > > > > > > collected.
> > > > > > > > > > > > The
> > > > > > > > > > > > > > > latter seems to be used for the new leader to
> > > > > determine the
> > > > > > > > > last
> > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > segment.
> > > > > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > topicPartition)
> > > > > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Right,.that is what we are currently doing. We
> > will
> > > > > update
> > > > > > > the
> > > > > > > > > > > > > > > javadocs and wiki with that. Earlier, we did not
> > want
> > > > > to
> > > > > > > remove
> > > > > > > > > > the
> > > > > > > > > > > > > > > segments which are not matched with leader epochs
> > > > from
> > > > > the
> > > > > > > > > ladder
> > > > > > > > > > > > > > > partition as they may be used later by a replica
> > > > which
> > > > > can
> > > > > > > > > > become a
> > > > > > > > > > > > > > > leader (unclean leader election) and refer those
> > > > > segments.
> > > > > > > But
> > > > > > > > > > that
> > > > > > > > > > > > > > > may leak these segments in remote storage until
> > the
> > > > > topic
> > > > > > > > > > lifetime.
> > > > > > > > > > > > We
> > > > > > > > > > > > > > > decided to cleanup the segments with the oldest
> > > > incase
> > > > > of
> > > > > > > size
> > > > > > > > > > based
> > > > > > > > > > > > > > > retention also.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it
> > > > seems
> > > > > > > that
> > > > > > > > > > they can
> > > > > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It will be useful to have metadata for RSM to
> > fetch
> > > > log
> > > > > > > > > segment.
> > > > > > > > > > It
> > > > > > > > > > > > > > > may create location/path using id with other
> > metadata
> > > > > too.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use
> > long
> > > > > > > instead of
> > > > > > > > > > Long?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Wanted to keep endPosition as optional to read
> > till
> > > > the
> > > > > > > end of
> > > > > > > > > > the
> > > > > > > > > > > > > > > segment and avoid sentinels.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > > > implementation
> > > > > > > > > > and
> > > > > > > > > > > > > > others
> > > > > > > > > > > > > > > Don't?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Actually,  RSM will not have any default
> > > > > implementations.
> > > > > > > > > Those 3
> > > > > > > > > > > > > > > methods were made default earlier for tests etc.
> > > > > Updated
> > > > > > > the
> > > > > > > > > > wiki.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102.4. Could we define
> > > > RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Sure, they will be added.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier
> > to
> > > > > pass
> > > > > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array
> > > > than
> > > > > a
> > > > > > > file
> > > > > > > > > > since
> > > > > > > > > > > > it
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Right, this is in plan.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it
> > > > needs
> > > > > > > both
> > > > > > > > > > > > baseOffset
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > startOffset. For example, deleteRecords() could
> > move
> > > > > the
> > > > > > > > > > startOffset
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > middle of a segment. If we copy the full segment
> > to
> > > > > remote
> > > > > > > > > > storage,
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Good point. startOffset is baseOffset by
> > default, if
> > > > > not
> > > > > > > set
> > > > > > > > > > > > explicitly.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Sure, updated the wiki.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > > > > 5103.1 Could we define the default value of
> > > > > non-required
> > > > > > > > > configs
> > > > > > > > > > > > (e.g the
> > > > > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Sure, that makes sense.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms
> > should
> > > > > > > default to
> > > > > > > > > > > > > > retention.ms,
> > > > > > > > > > > > > > > instead of remote.log.retention.minutes.
> > Similarly,
> > > > it
> > > > > > > seems
> > > > > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > > > > segment.bytes.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Right, we do not have  remote.log.retention as we
> > > > > discussed
> > > > > > > > > > earlier.
> > > > > > > > > > > > > > > Thanks for catching the typo.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > > > description
> > > > > > > > > says
> > > > > > > > > > > > "used in
> > > > > > > > > > > > > > > scheduling tasks to copy segments, fetch remote
> > log
> > > > > > > indexes and
> > > > > > > > > > > > clean up
> > > > > > > > > > > > > > > remote log segments". However, there is a
> > separate
> > > > > > > > > > > > > > > config remote.log.reader.threads for fetching
> > remote
> > > > > data.
> > > > > > > It's
> > > > > > > > > > > > weird to
> > > > > > > > > > > > > > > fetch remote index and log in different thread
> > pools
> > > > > since
> > > > > > > both
> > > > > > > > > > are
> > > > > > > > > > > > used
> > > > > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Right, remote.log.manager.thread.pool is mainly
> > used
> > > > > for
> > > > > > > > > > copy/cleanup
> > > > > > > > > > > > > > > activities. Fetch path always goes through
> > > > > > > > > > remote.log.reader.threads.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is
> > that
> > > > > the
> > > > > > > amount
> > > > > > > > > > of
> > > > > > > > > > > > time
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > back off when there is no work to do? If so,
> > perhaps
> > > > it
> > > > > > > can be
> > > > > > > > > > > > renamed as
> > > > > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > This is the delay interval for each iteration.
> > It may
> > > > > be
> > > > > > > > > renamed
> > > > > > > > > > to
> > > > > > > > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > > > > rlm_retry_interval_ms
> > > > > > > > > > > > configs? If
> > > > > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > remote.log.manager.task.interval.ms is the
> > process
> > > > > > > internal,
> > > > > > > > > > retry
> > > > > > > > > > > > > > > interval is missing in the configs, which will be
> > > > > updated
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly
> > LRU) of
> > > > > the
> > > > > > > index
> > > > > > > > > > > > files of
> > > > > > > > > > > > > > > remote log segments to avoid multiple index
> > fetches
> > > > > from
> > > > > > > the
> > > > > > > > > > remote
> > > > > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on
> > > > disk,
> > > > > > > where is
> > > > > > > > > > it
> > > > > > > > > > > > > > stored?
> > > > > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It is stored on disk. They are stored in a
> > directory
> > > > > > > > > > > > > > > `remote-log-index-cache` under log dir. We plan
> > to
> > > > > have a
> > > > > > > > > config
> > > > > > > > > > for
> > > > > > > > > > > > > > > that instead of default. We will have a
> > configuration
> > > > > for
> > > > > > > that.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> > > > Earliest
> > > > > > > Local
> > > > > > > > > > Offset
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > different places. It would be useful to
> > standardize
> > > > the
> > > > > > > > > > terminology.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Sure.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux
> > state".
> > > > > It
> > > > > > > listed
> > > > > > > > > > two
> > > > > > > > > > > > > > options
> > > > > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > > > > We already mentioned in the KIP that we chose
> > > > option-2.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5107. Follower to leader transition: It has step
> > 2,
> > > > > but not
> > > > > > > > > step
> > > > > > > > > > 1.
> > > > > > > > > > > > > > > Step-1 is there but it is not explicitly
> > highlighted.
> > > > > It is
> > > > > > > > > > previous
> > > > > > > > > > > > > > > table to step-2.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5108. If a consumer fetches from the remote data
> > and
> > > > > the
> > > > > > > remote
> > > > > > > > > > > > storage
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > not available, what error code is used in the
> > fetch
> > > > > > > response?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Good point. We have not yet defined the error for
> > > > this
> > > > > > > case. We
> > > > > > > > > > need
> > > > > > > > > > > > > > > to define an error message and send the same in
> > fetch
> > > > > > > response.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it
> > returns
> > > > the
> > > > > > > first
> > > > > > > > > > message
> > > > > > > > > > > > > > > offset whose timestamp is >= to the given
> > timestamp
> > > > in
> > > > > the
> > > > > > > > > > request.
> > > > > > > > > > > > That
> > > > > > > > > > > > > > > means it checks in remote log time indexes first,
> > > > after
> > > > > > > which
> > > > > > > > > > local
> > > > > > > > > > > > log
> > > > > > > > > > > > > > > time indexes are checked." Could you document
> > which
> > > > > method
> > > > > > > in
> > > > > > > > > > RLMM is
> > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > for this?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Okay.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> > > > segment
> > > > > > > metadata
> > > > > > > > > > of
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > partition with a delete marker and publishes
> > them to
> > > > > RLMM."
> > > > > > > > > This
> > > > > > > > > > > > seems
> > > > > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset
> > for
> > > > the
> > > > > > > > > earliest
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > > epoch by calling
> > > > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > > int
> > > > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > > > > offset." Do
> > > > > > > we
> > > > > > > > > > need
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > since replication propagates logStartOffset
> > already?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Good point. Right, existing replication protocol
> > > > takes
> > > > > > > care of
> > > > > > > > > > > > > > > updating the followers’s log start offset
> > received
> > > > > from the
> > > > > > > > > > leader.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough
> > for
> > > > > fetching
> > > > > > > > > from
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > storage?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Remote reads may fail within the current default
> > wait
> > > > > > > time, but
> > > > > > > > > > > > > > > subsequent fetches would be able to serve as that
> > > > data
> > > > > is
> > > > > > > > > stored
> > > > > > > > > > in
> > > > > > > > > > > > > > > the local cache. This cache is currently
> > implemented
> > > > in
> > > > > > > RSMs.
> > > > > > > > > > But we
> > > > > > > > > > > > > > > plan to pull this into the remote log messaging
> > layer
> > > > > in
> > > > > > > > > future.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5113. "Committed offsets can be stored in a local
> > > > file
> > > > > to
> > > > > > > avoid
> > > > > > > > > > > > reading
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > messages again when a broker is restarted."
> > Could you
> > > > > > > describe
> > > > > > > > > > the
> > > > > > > > > > > > format
> > > > > > > > > > > > > > > and the location of the file? Also, could the
> > same
> > > > > message
> > > > > > > be
> > > > > > > > > > > > processed
> > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> > > > > handle
> > > > > > > that?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > > > > 5114.1 There are two records named
> > > > > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > > > > with
> > > > > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > > > > document
> > > > > > > > > whether
> > > > > > > > > > > > > > endOffset
> > > > > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> > > > > explain
> > > > > > > > > > LeaderEpoch
> > > > > > > > > > > > a
> > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > more? Is that the epoch of the leader when it
> > copies
> > > > > the
> > > > > > > > > segment
> > > > > > > > > > to
> > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Right, this is the leader epoch of the broker
> > which
> > > > > copied
> > > > > > > this
> > > > > > > > > > > > > > > segment. This is helpful in reason about which
> > broker
> > > > > > > copied
> > > > > > > > > the
> > > > > > > > > > > > > > > segment to remote storage.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a
> > bit
> > > > > more?
> > > > > > > Each
> > > > > > > > > > > > record in
> > > > > > > > > > > > > > > Kafka already has a timestamp field. Could we
> > just
> > > > use
> > > > > > > that?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > This is the  timestamp at which  the respective
> > event
> > > > > > > occurred.
> > > > > > > > > > Added
> > > > > > > > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can
> > be  any
> > > > > other
> > > > > > > > > > > > > > > implementation. We thought about that but it
> > looked
> > > > > > > cleaner to
> > > > > > > > > > use at
> > > > > > > > > > > > > > > the message structure level instead of getting
> > that
> > > > > from
> > > > > > > the
> > > > > > > > > > consumer
> > > > > > > > > > > > > > > record and using that to build the respective
> > event.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be
> > int32?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Right, it looks like config allows only int
> > value >=
> > > > > 14.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be
> > confused
> > > > > with
> > > > > > > the
> > > > > > > > > log
> > > > > > > > > > > > cleaner
> > > > > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth
> > like
> > > > > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I am fine with RemotePartitionRemover or
> > > > > > > > > > RemoteLogDeletionManager(we
> > > > > > > > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked
> > and
> > > > > > > processes
> > > > > > > > > it
> > > > > > > > > > if
> > > > > > > > > > > > it is
> > > > > > > > > > > > > > > not yet processed earlier." How does it know
> > whether
> > > > > > > > > > > > > > > delete_partition_marked has been processed
> > earlier?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > This is to handle duplicate
> > delete_partition_marked
> > > > > > > events. RLC
> > > > > > > > > > > > > > > internally maintains a state for the
> > delete_partition
> > > > > > > events
> > > > > > > > > and
> > > > > > > > > > if
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > already has an existing event then it ignores if
> > it
> > > > is
> > > > > > > already
> > > > > > > > > > being
> > > > > > > > > > > > > > > processed.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5117. Should we add a new MessageFormatter to
> > read
> > > > the
> > > > > tier
> > > > > > > > > > metadata
> > > > > > > > > > > > > > topic?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Right, this is in plan but did not mention it in
> > the
> > > > > KIP.
> > > > > > > This
> > > > > > > > > > will
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > useful for debugging purposes too.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task
> > > > queue
> > > > > > > size.
> > > > > > > > > If
> > > > > > > > > > the
> > > > > > > > > > > > task
> > > > > > > > > > > > > > > queue is full, broker will stop reading remote
> > log
> > > > > > > segments."
> > > > > > > > > > What
> > > > > > > > > > > > do we
> > > > > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We return an error response for that partition.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5119. It would be useful to list all things not
> > > > > supported
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > first
> > > > > > > > > > > > > > > version in a Future work or Limitations section.
> > For
> > > > > > > example,
> > > > > > > > > > > > compacted
> > > > > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable
> > from
> > > > > true
> > > > > > > to
> > > > > > > > > > false,
> > > > > > > > > > > > etc.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We already have a non-goals section which is
> > filled
> > > > > with
> > > > > > > some
> > > > > > > > > of
> > > > > > > > > > > > these
> > > > > > > > > > > > > > > details. Do we need another limitations section?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <
> > > > > jun@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > > > > below.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 605.2 "Build the local leader epoch cache by
> > > > cutting
> > > > > the
> > > > > > > > > leader
> > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > > sequence received from remote storage to [LSO,
> > > > > ELO]." I
> > > > > > > > > > mentioned
> > > > > > > > > > > > an
> > > > > > > > > > > > > > issue
> > > > > > > > > > > > > > > > earlier. Suppose the leader's local start
> > offset is
> > > > > 100.
> > > > > > > The
> > > > > > > > > > > > follower
> > > > > > > > > > > > > > finds
> > > > > > > > > > > > > > > > a remote segment covering offset range [80,
> > 120).
> > > > The
> > > > > > > > > > producerState
> > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > this remote segment is up to offset 120. To
> > trim
> > > > the
> > > > > > > > > > producerState
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > offset 100 requires more work since one needs
> > to
> > > > > > > download the
> > > > > > > > > > > > previous
> > > > > > > > > > > > > > > > producerState up to offset 80 and then replay
> > the
> > > > > > > messages
> > > > > > > > > > from 80
> > > > > > > > > > > > to
> > > > > > > > > > > > > > 100.
> > > > > > > > > > > > > > > > It seems that it's simpler in this case for the
> > > > > follower
> > > > > > > just
> > > > > > > > > > to
> > > > > > > > > > > > take
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > remote segment as it is and start fetching from
> > > > > offset
> > > > > > > 120.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It
> > > > seems
> > > > > that
> > > > > > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only
> > called
> > > > > on the
> > > > > > > > > > replicas
> > > > > > > > > > > > for
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > > > obtains the metadata for remote segments for
> > > > > deletion.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > > > > implemented
> > > > > > > now.
> > > > > > > > > > > > Could you
> > > > > > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how
> > the
> > > > > > > following
> > > > > > > > > two
> > > > > > > > > > > > APIs
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > used. According to the wiki, the former is
> > used for
> > > > > topic
> > > > > > > > > > deletion
> > > > > > > > > > > > and
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > latter is used for retention. It seems that
> > > > retention
> > > > > > > should
> > > > > > > > > > use
> > > > > > > > > > > > the
> > > > > > > > > > > > > > former
> > > > > > > > > > > > > > > > since remote segments without a matching epoch
> > in
> > > > the
> > > > > > > leader
> > > > > > > > > > > > > > (potentially
> > > > > > > > > > > > > > > > due to unclean leader election) also need to be
> > > > > garbage
> > > > > > > > > > collected.
> > > > > > > > > > > > The
> > > > > > > > > > > > > > > > latter seems to be used for the new leader to
> > > > > determine
> > > > > > > the
> > > > > > > > > > last
> > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > segment.
> > > > > > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > topicPartition)
> > > > > > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(),
> > it
> > > > > seems
> > > > > > > that
> > > > > > > > > > they
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > > > > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use
> > long
> > > > > > > instead
> > > > > > > > > of
> > > > > > > > > > > > Long?
> > > > > > > > > > > > > > > > 5102.3 Why only some of the methods have
> > default
> > > > > > > > > > implementation and
> > > > > > > > > > > > > > others
> > > > > > > > > > > > > > > > don't?
> > > > > > > > > > > > > > > > 5102.4. Could we define
> > > > > RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's
> > easier to
> > > > > pass
> > > > > > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte
> > array
> > > > > than a
> > > > > > > file
> > > > > > > > > > > > since it
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that
> > it
> > > > > needs
> > > > > > > both
> > > > > > > > > > > > > > baseOffset and
> > > > > > > > > > > > > > > > startOffset. For example, deleteRecords() could
> > > > move
> > > > > the
> > > > > > > > > > > > startOffset
> > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > middle of a segment. If we copy the full
> > segment to
> > > > > > > remote
> > > > > > > > > > > > storage, the
> > > > > > > > > > > > > > > > baseOffset and the startOffset will be
> > different.
> > > > > > > > > > > > > > > > 5102.7 Could we define all the public methods
> > for
> > > > > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > > > > > 5103.1 Could we define the default value of
> > > > > non-required
> > > > > > > > > > configs
> > > > > > > > > > > > (e.g
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms
> > should
> > > > > > > default
> > > > > > > > > to
> > > > > > > > > > > > > > retention.ms,
> > > > > > > > > > > > > > > > instead of remote.log.retention.minutes.
> > Similarly,
> > > > > it
> > > > > > > seems
> > > > > > > > > > > > > > > > that local.log.retention.bytes should default
> > to
> > > > > > > > > segment.bytes.
> > > > > > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > > > > > description
> > > > > > > > > > says
> > > > > > > > > > > > "used
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > scheduling tasks to copy segments, fetch
> > remote log
> > > > > > > indexes
> > > > > > > > > and
> > > > > > > > > > > > clean
> > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > remote log segments". However, there is a
> > separate
> > > > > > > > > > > > > > > > config remote.log.reader.threads for fetching
> > > > remote
> > > > > > > data.
> > > > > > > > > It's
> > > > > > > > > > > > weird
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > fetch remote index and log in different thread
> > > > pools
> > > > > > > since
> > > > > > > > > > both are
> > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is
> > > > that
> > > > > the
> > > > > > > > > > amount of
> > > > > > > > > > > > > > time to
> > > > > > > > > > > > > > > > back off when there is no work to do? If so,
> > > > perhaps
> > > > > it
> > > > > > > can
> > > > > > > > > be
> > > > > > > > > > > > renamed
> > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > > > > rlm_retry_interval_ms
> > > > > > > > > > > > configs?
> > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly
> > LRU)
> > > > > of the
> > > > > > > > > index
> > > > > > > > > > > > files
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > remote log segments to avoid multiple index
> > fetches
> > > > > from
> > > > > > > the
> > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage." Is the RLM in memory or on disk? If
> > on
> > > > > disk,
> > > > > > > where
> > > > > > > > > > is it
> > > > > > > > > > > > > > stored?
> > > > > > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> > > > > Earliest
> > > > > > > Local
> > > > > > > > > > > > Offset in
> > > > > > > > > > > > > > > > different places. It would be useful to
> > standardize
> > > > > the
> > > > > > > > > > > > terminology.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux
> > > > > state". It
> > > > > > > > > > listed two
> > > > > > > > > > > > > > options
> > > > > > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5107. Follower to leader transition: It has
> > step 2,
> > > > > but
> > > > > > > not
> > > > > > > > > > step 1.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5108. If a consumer fetches from the remote
> > data
> > > > and
> > > > > the
> > > > > > > > > remote
> > > > > > > > > > > > > > storage is
> > > > > > > > > > > > > > > > not available, what error code is used in the
> > fetch
> > > > > > > response?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it
> > returns
> > > > > the
> > > > > > > first
> > > > > > > > > > > > message
> > > > > > > > > > > > > > > > offset whose timestamp is >= to the given
> > timestamp
> > > > > in
> > > > > > > the
> > > > > > > > > > request.
> > > > > > > > > > > > > > That
> > > > > > > > > > > > > > > > means it checks in remote log time indexes
> > first,
> > > > > after
> > > > > > > which
> > > > > > > > > > > > local log
> > > > > > > > > > > > > > > > time indexes are checked." Could you document
> > which
> > > > > > > method in
> > > > > > > > > > RLMM
> > > > > > > > > > > > is
> > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > for this?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> > > > > segment
> > > > > > > > > > metadata of
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > partition with a delete marker and publishes
> > them
> > > > to
> > > > > > > RLMM."
> > > > > > > > > > This
> > > > > > > > > > > > seems
> > > > > > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5111. "RLM follower fetches the earliest
> > offset for
> > > > > the
> > > > > > > > > > earliest
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > epoch by calling
> > > > > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > > int
> > > > > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > > > > offset."
> > > > > > > Do we
> > > > > > > > > > need
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > since replication propagates logStartOffset
> > > > already?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough
> > for
> > > > > > > fetching
> > > > > > > > > > from
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5113. "Committed offsets can be stored in a
> > local
> > > > > file to
> > > > > > > > > avoid
> > > > > > > > > > > > > > reading the
> > > > > > > > > > > > > > > > messages again when a broker is restarted."
> > Could
> > > > you
> > > > > > > > > describe
> > > > > > > > > > the
> > > > > > > > > > > > > > format
> > > > > > > > > > > > > > > > and the location of the file? Also, could the
> > same
> > > > > > > message be
> > > > > > > > > > > > > > processed by
> > > > > > > > > > > > > > > > RLMM again after broker restart? If so, how do
> > we
> > > > > handle
> > > > > > > > > that?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > > > > > 5114.1 There are two records named
> > > > > > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > > > > with
> > > > > > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > > > > document
> > > > > > > > > > whether
> > > > > > > > > > > > > > endOffset
> > > > > > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could
> > you
> > > > > explain
> > > > > > > > > > > > LeaderEpoch a
> > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > more? Is that the epoch of the leader when it
> > > > copies
> > > > > the
> > > > > > > > > > segment to
> > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this
> > a bit
> > > > > more?
> > > > > > > > > Each
> > > > > > > > > > > > record
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > Kafka already has a timestamp field. Could we
> > just
> > > > > use
> > > > > > > that?
> > > > > > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be
> > > > int32?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be
> > confused
> > > > > with
> > > > > > > the
> > > > > > > > > > log
> > > > > > > > > > > > > > cleaner
> > > > > > > > > > > > > > > > for compaction. Perhaps it can be renamed to
> > sth
> > > > like
> > > > > > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5116. "RLC receives the
> > delete_partition_marked and
> > > > > > > processes
> > > > > > > > > > it
> > > > > > > > > > > > if it
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > not yet processed earlier." How does it know
> > > > whether
> > > > > > > > > > > > > > > > delete_partition_marked has been processed
> > earlier?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5117. Should we add a new MessageFormatter to
> > read
> > > > > the
> > > > > > > tier
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > topic?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5118. "Maximum remote log reader thread pool
> > task
> > > > > queue
> > > > > > > size.
> > > > > > > > > > If
> > > > > > > > > > > > the
> > > > > > > > > > > > > > task
> > > > > > > > > > > > > > > > queue is full, broker will stop reading remote
> > log
> > > > > > > segments."
> > > > > > > > > > What
> > > > > > > > > > > > do
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5119. It would be useful to list all things not
> > > > > > > supported in
> > > > > > > > > > the
> > > > > > > > > > > > first
> > > > > > > > > > > > > > > > version in a Future work or Limitations
> > section.
> > > > For
> > > > > > > example,
> > > > > > > > > > > > compacted
> > > > > > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable
> > > > from
> > > > > > > true to
> > > > > > > > > > false,
> > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik
> > Prakasam <
> > > > > > > > > > > > > > kprakasam@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.duggana@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.duggana@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
> > > > <
> > > > > > > > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > > > > > > > (
> > > > > > > > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > > > > > > > kprakasam@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.ch@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 (
> > ch@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.duggana@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
> > > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > >
> > > >
> >

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

Posted by Jun Rao <ju...@confluent.io>.
Hi, Satish,

Thanks for the reply. Just a couple of more comments.

9300. Could we add a section on downgrade?

9301. "Feature test cases and test results are documented in this google
spreadsheet
<https://docs.google.com/spreadsheets/d/1XhNJqjzwXvMCcAOhEH0sSXU6RTvyoSf93DHF-YMfGLk/edit?usp=sharing>
." The google spreadsheet doesn't have a link. Could we add the plan for
integration and system systems?

Jun

On Mon, Feb 8, 2021 at 8:21 AM Satish Duggana <sa...@gmail.com>
wrote:

> Hi Manikumar,
> Thanks for your comment. Please find the inline replies below.
>
> 9201. Can we make it clear by mentioning RLM as an internal component and
> RLMM and RSM are pluggable components.
> It's also good to update/add the RLM, RLMM components  diagrams with
> internal components/tasks.
>
> Updated the KIP.
>
> 9202. RLMM metadata topic is a non-compact topic. Can we make clear that
> metadata topic retention time should be greater
>  than user topics. or How about setting retention set to -1 (unlimited) and
> let users to modify it.
>
> I am fine with -1 and updated the configuration doc and made it clear
> in the KIP.
>
> 9203. RLMM has additional responsibility of handling topic delete requests.
> With current design, RLMM implementations need
> to implement deletion logic. How about making RemotePartitionRemover as a
> separate task, which can be reused..
>
> Good point. We had plans to address that in a future KIP, added in
> future work items.
>
> 9204. Can we list all the new error codes added by the KIP?
>
> Those are mentioned in the protocol changes section.
>
> 9205. Can we also support  `remote.log.metadata.producr.*`,
> `remote.log.metadata.consumer.*` prefixed configs to
> override any default configs ?. also include RLMM cache configs, if any.
>
> Good point. Updated the KIP.
>
> 9206. In the Upgrade section, we have below statement
> "Before enabling tiered storage, you should make sure the producer
> snapshots are built for all the segments for that topic
> in all followers. You should wait till the log retention occurs for all the
> segments so that all the segments have producer snapshots."
> Can you add more details about this constraint?
>
> Sure, updated in the KIP.
>
>
> ~Satish.
>
> On Tue, 2 Feb 2021 at 21:04, Manikumar <ma...@gmail.com> wrote:
> >
> > Hi Satish,
> >
> > Thanks for the KIP.  some of my comments below.
> >
> > 9201. Can we make it clear by mentioning RLM as an internal component and
> > RLMM and RSM are pluggable components.
> > It's also good to update/add the RLM, RLMM components  diagrams with
> > internal components/tasks.
> >
> > 9202. RLMM metadata topic is a non-compact topic. Can we make clear that
> > metadata topic retention time should be greater
> >  than user topics. or How about setting retention set to -1 (unlimited)
> and
> > let users to modify it.
> >
> > 9203. RLMM has additional responsibility of handling topic delete
> requests.
> > With current design, RLMM implementations need
> > to implement deletion logic. How about making RemotePartitionRemover as a
> > separate task, which can be reused..
> >
> > 9204. Can we list all the new error codes added by the KIP?
> >
> > 9205. Can we also support  `remote.log.metadata.producr.*`,
> > `remote.log.metadata.consumer.*` prefixed configs to
> > override any default configs ?. also include RLMM cache configs, if any.
> >
> > 9206. In the Upgrade section, we have below statement
> > "Before enabling tiered storage, you should make sure the producer
> > snapshots are built for all the segments for that topic
> > in all followers. You should wait till the log retention occurs for all
> the
> > segments so that all the segments have producer snapshots."
> >
> >  Can you add more details about this constraint?
> >
> >
> > Thanks,
> > Manikumar
> >
> > On Tue, Jan 26, 2021 at 1:12 AM Jun Rao <ju...@confluent.io> wrote:
> >
> > > Hi, Satish,
> > >
> > > Thanks for the reply. A few more comments below.
> > >
> > > 6000. When RSM/RLMM is not available, it seems that we need to return a
> > > retriable error to the affected client requests. So, should we add a
> new
> > > error code for requests like fetch and listOffset?
> > >
> > > 9111. RemotePartitionRemover:
> > > 91111.1 Could we make it clear that it's part of the default RLMM
> > > implementation and a separate implementation is needed if one has a
> > > customized RLMM?
> > > 91111.2 "RPRM gets all the remote log segments for the partition using
> RLMM
> > > and each of these remote log segments is deleted with the next
> steps.RLMM
> > > subscribes to the local remote log metadata partitions and it will
> have the
> > > segment metadata of all the user topic partitions associated with that
> > > remote log metadata partition." It seems that RLMM needs to subscribe
> to
> > > the remote log metadata partitions first before those segments can be
> > > deleted?
> > > 9111.3 There are still references to "remote log cleaners". They need
> to be
> > > replaced with RemotePartitionRemover.
> > >
> > > 9114.1 Could we add the requirement on log.message.format before
> enabling
> > > tiered storage?
> > >
> > > 9116. RemoteLogMetadataFormatter: This is used with the ConsoleConsumer
> > > tool, right? Then, are those new options for ConsoleConsumer and how
> are
> > > they passed in?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Mon, Jan 25, 2021 at 8:02 AM Satish Duggana <
> satish.duggana@gmail.com>
> > > wrote:
> > >
> > > > Hi Jun,
> > > >
> > > > Thanks for your comments. Please find the inline replies below.
> > > > Several of these were discussed/clarified in our last discussion.
> > > >
> > > > 6000. Since we are returning new error codes, we need to bump up the
> > > > protocol version for Fetch request. Also, it will be useful to
> document
> > > all
> > > > new error codes and whether they are retriable or not.
> > > > This previous comment doesn't seem to be addressed.
> > > >
> > > > Updated KIP on fetch protocol and error code.
> > > >
> > > > 9110. flat_file_format: Could you define the serialized
> representation
> > > for
> > > > each field in the header?
> > > >
> > > > Added the serialized representation types in the KIP.
> > > >
> > > > 9111. RPM has the following 2 steps.
> > > > "1. The controller publishes delete_partition_marked event to say
> that
> > > the
> > > > partition is marked for deletion. There can be multiple events
> published
> > > > when the controller restarts or failover and this event will be
> > > > deduplicated by RPM.
> > > > 2. RPM receives the delete_partition_marked and processes it if it
> is not
> > > > yet processed earlier."
> > > > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> > > default
> > > > RLMM implementation or is it meant for any RLMM implementation?
> > > >
> > > > RPM(or RPRM) is only for the default RLMM implementation. RPM
> receives
> > > > the delete_partition_marked events from RLMM and acts on them.
> Updated
> > > > KIP with more details.
> > > >
> > > > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful
> to
> > > make
> > > > it clear in the comment that this is for the initial retry backoff.
> > > >
> > > > Updated the KIP.
> > > >
> > > > 9113. RLMM:
> > > > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> > > updated.
> > > >
> > > > Updated javadoc of this method.
> > > >
> > > > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch
> field
> > > > since all other update events have leaderEpoch?
> > > >
> > > > +1 to have that,  updated the KIP.
> > > >
> > > > 9113.3 Could we rename RemotePartitionState to
> RemotePartitionDeleteState
> > > > to make it clear this is for deletion?
> > > >
> > > > Sure, updated the KIP.
> > > >
> > > > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to
> make it
> > > > clear this is for segment?
> > > >
> > > > Sure, Updated the KIP.
> > > >
> > > > 9114.Upgrade:
> > > > 9114.1 It seems that we require message format > 0.11 for turning on
> the
> > > > remote store feature.
> > > >
> > > > 9114.2 It's not clear to me why remote.log.storage.system.enable
> needs to
> > > > be set to true before bumping up inter.broker.protocol.version.
> > > >
> > > > Agree that based on our discussion, this is not required. Upgrade
> > > > notes is updated.
> > > >
> > > > 9114.3 "If the topic-id is not received in the LeaderAndIsr request
> then
> > > > remote log storage will not start. But it will log an error message
> in
> > > the
> > > > log. One way to address this is to do a rolling restart of that
> broker,
> > > so
> > > > that the leader will be moved to another broker and the controller
> will
> > > > send LeaderAndIsr with the registered topic-id." Why is this needed?
> With
> > > > KIP-516, after upgrading to the latest protocol, topicIds are auto
> > > created.
> > > >
> > > > I thought there may be edge cases of not receiving topic-id but you
> > > > clarified in the meeting that won’t be the case. I agree that it is
> > > > not needed.
> > > >
> > > > 9115. testing: Could you comment on how we plan to do integration and
> > > > system tests? Do we plan to include, for example, an in-memory
> > > > implementation of RSM?
> > > >
> > > > For integration tests, we can have file based RSM that we have. For
> > > > system tests, we can have a single node HDFS cluster in one of the
> > > > containers and use HDFS RSM implementation.
> > > >
> > > > 9116. There is still a todo for the message formatter.
> > > >
> > > > Updated the KIP with the format.
> > > >
> > > >
> > > > Satish.
> > > >
> > > >
> > > > On Tue, 12 Jan 2021 at 07:07, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the reply. A few more followup comments.
> > > > >
> > > > > 6000. Since we are returning new error codes, we need to bump up
> the
> > > > > protocol version for Fetch request. Also, it will be useful to
> document
> > > > all
> > > > > new error codes and whether they are retriable or not.
> > > > > This previous comment doesn't seem to be addressed.
> > > > >
> > > > > 9110. flat_file_format: Could you define the serialized
> representation
> > > > for
> > > > > each field in the header?
> > > > >
> > > > > 9111. RPM has the following 2 steps.
> > > > > "1. The controller publishes delete_partition_marked event to say
> that
> > > > the
> > > > > partition is marked for deletion. There can be multiple events
> > > published
> > > > > when the controller restarts or failover and this event will be
> > > > > deduplicated by RPM.
> > > > > 2. RPM receives the delete_partition_marked and processes it if it
> is
> > > not
> > > > > yet processed earlier."
> > > > > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> > > > default
> > > > > RLMM implementation or is it meant for any RLMM implementation?
> > > > >
> > > > > 9112. remote.log.manager.task.retry.backoff.ms: It would be
> useful to
> > > > make
> > > > > it clear in the comment that this is for the initial retry backoff.
> > > > >
> > > > > 9113. RLMM:
> > > > > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> > > > updated.
> > > > > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch
> field
> > > > > since all other update events have leaderEpoch?
> > > > > 9113.3 Could we rename RemotePartitionState to
> > > RemotePartitionDeleteState
> > > > > to make it clear this is for deletion?
> > > > > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to
> make
> > > it
> > > > > clear this is for segment?
> > > > >
> > > > > 9114.Upgrade:
> > > > > 9114.1 It seems that we require message format > 0.11 for turning
> on
> > > the
> > > > > remote store feature.
> > > > > 9114.2 It's not clear to me why remote.log.storage.system.enable
> needs
> > > to
> > > > > be set to true before bumping up inter.broker.protocol.version.
> > > > > 9114.3 "If the topic-id is not received in the LeaderAndIsr request
> > > then
> > > > > remote log storage will not start. But it will log an error
> message in
> > > > the
> > > > > log. One way to address this is to do a rolling restart of that
> broker,
> > > > so
> > > > > that the leader will be moved to another broker and the controller
> will
> > > > > send LeaderAndIsr with the registered topic-id." Why is this
> needed?
> > > With
> > > > > KIP-516, after upgrading to the latest protocol, topicIds are auto
> > > > created.
> > > > >
> > > > > 9115. testing: Could you comment on how we plan to do integration
> and
> > > > > system tests? Do we plan to include, for example, an in-memory
> > > > > implementation of RSM?
> > > > >
> > > > > 9116. There is still a todo for the message formatter.
> > > > >
> > > > > Jun
> > > > >
> > > > > On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <
> > > satish.duggana@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > >
> > > > > > 6022. For packages used for server plugins, the convention is to
> > > > > > use org.apache.kafka.server. See java-based Authorizer as an
> example.
> > > > > >
> > > > > > Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> > > > > > ‘org.apache.kafka.server.log.remote.storage’.  Updated in the
> KIP.
> > > > > >
> > > > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format?
> The
> > > > flat
> > > > > > file captures the state of the remote segments. After a
> partition is
> > > > > > deleted, it seems that we just need to remove the partitions's
> remote
> > > > > > segments from the flat file.
> > > > > >
> > > > > > DeletePartitionState might not yet have been processed by RPM
> and not
> > > > > > completed. We  will not have  that in flat  file format  once it
> > > > > > reaches DELETE_PARTITION_FINISHED state.
> > > > > >
> > > > > > 9101. Upgrade: It will be useful to allow direct upgrade from an
> old
> > > > > > version. It seems that's doable. One can just do the normal
> upgrade
> > > > first
> > > > > > and wait enough time (for producer snapshots to be built), and
> then
> > > > enable
> > > > > > remote storage.
> > > > > >
> > > > > > Upgrade notes updates in the KIP.
> > > > > >
> > > > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM
> starts
> > > > > > tracking the remote segments when
> RLMM.onPartitionLeadershipChanges()
> > > > is
> > > > > > called with the broker being the leader for __remote_log_metadata
> > > > > > partition? If so, could we document it?
> > > > > >
> > > > > > The current plan is to have that as part of RLMM and RPM uses
> that to
> > > > > > get the remote segments list. I will add this detail in the
> > > respective
> > > > > > sections.
> > > > > >
> > > > > > Satish.
> > > > > >
> > > > > > On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> > > > > > >
> > > > > > > Hi, Satish,
> > > > > > >
> > > > > > > Thanks for the reply. A few more followup comments.
> > > > > > >
> > > > > > > 6022. For packages used for server plugins, the convention is
> to
> > > > > > > use org.apache.kafka.server. See java-based Authorizer as an
> > > example.
> > > > > > >
> > > > > > > 9100. Do we need DeletePartitionStateRecord in
> flat_file_format?
> > > The
> > > > flat
> > > > > > > file captures the state of the remote segments. After a
> partition
> > > is
> > > > > > > deleted, it seems that we just need to remove the partitions's
> > > remote
> > > > > > > segments from the flat file.
> > > > > > >
> > > > > > > 9101. Upgrade: It will be useful to allow direct upgrade from
> an
> > > old
> > > > > > > version. It seems that's doable. One can just do the normal
> upgrade
> > > > first
> > > > > > > and wait enough time (for producer snapshots to be built), and
> then
> > > > > > enable
> > > > > > > remote storage.
> > > > > > >
> > > > > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM
> > > starts
> > > > > > > tracking the remote segments when
> > > > RLMM.onPartitionLeadershipChanges() is
> > > > > > > called with the broker being the leader for
> __remote_log_metadata
> > > > > > > partition? If so, could we document it?
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <
> > > > kprakasam@confluent.io
> > > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Satish,
> > > > > > > >
> > > > > > > > Thanks for the updates! A few more comments below.
> > > > > > > >
> > > > > > > > 9001. Under the "Upgrade" section, there is a line
> mentioning:
> > > > > > "Upgrade the
> > > > > > > > existing Kafka cluster to 2.7 version and allow this to run
> for
> > > > the log
> > > > > > > > retention of user topics that you want to enable tiered
> storage.
> > > > This
> > > > > > will
> > > > > > > > allow all the topics to have the producer snapshots
> generated for
> > > > each
> > > > > > log
> > > > > > > > segment." -- Which associated change in AK were you
> referring to
> > > > here?
> > > > > > Is
> > > > > > > > it: https://github.com/apache/kafka/pull/7929 ? It seems
> like I
> > > > don't
> > > > > > see
> > > > > > > > it in the 2.7 release branch yet, here is the link:
> > > > > > > > https://github.com/apache/kafka/commits/2.7.
> > > > > > > >
> > > > > > > > 9002. Under the "Upgrade" section, the configuration
> mentioned is
> > > > > > > > 'remote.log.storage.system.enable'. However, under "Public
> > > > Interfaces"
> > > > > > > > section the corresponding configuration is
> > > > > > 'remote.storage.system.enable'.
> > > > > > > > Could we use the same one in both, maybe
> > > > > > > > 'remote.log.storage.system.enable'?
> > > > > > > >
> > > > > > > > 9003. Under "Per Topic Configuration", the KIP recommends
> setting
> > > > > > > > 'remote.log.storage.enable' to true at a per-topic level. It
> will
> > > > be
> > > > > > useful
> > > > > > > > to add a line that if the user wants to enable it for all
> topics,
> > > > then
> > > > > > they
> > > > > > > > should be able to set the cluster-wide default to true.
> Also, it
> > > > will
> > > > > > be
> > > > > > > > useful to mention that the KIP currently does not support
> setting
> > > > it to
> > > > > > > > false (after it is set to true), and add that to the future
> work
> > > > > > section.
> > > > > > > >
> > > > > > > > 9004. Under "Committed offsets file format", the sample
> provided
> > > > shows
> > > > > > > > partition number and offset. Is the topic name required for
> > > > identifying
> > > > > > > > which topic the partitions belong to?
> > > > > > > >
> > > > > > > > 9005. Under "Internal flat-file store format of remote log
> > > > metadata",
> > > > > > it
> > > > > > > > seems useful to specify both topic name and topic ID for
> > > debugging
> > > > > > > > purposes.
> > > > > > > >
> > > > > > > > 9006. Under "Internal flat-file store format of remote log
> > > > metadata",
> > > > > > the
> > > > > > > > description of "metadata-topic-offset" currently says
> "offset of
> > > > the
> > > > > > remote
> > > > > > > > log metadata topic from which this topic partition's remote
> log
> > > > > > metadata is
> > > > > > > > fetched." Just for the wording, perhaps you meant to refer
> to the
> > > > > > offset
> > > > > > > > upto which the file has been committed? i.e. "offset of the
> > > remote
> > > > log
> > > > > > > > metadata topic upto which this topic partition's remote log
> > > > metadata
> > > > > > has
> > > > > > > > been committed into this file."
> > > > > > > >
> > > > > > > > 9007. Under "Internal flat-file store format of remote log
> > > > metadata",
> > > > > > the
> > > > > > > > schema of the payload (i.e. beyond the header) seems to
> contain
> > > the
> > > > > > events
> > > > > > > > from the metadata topic. It seems useful to instead persist
> the
> > > > > > > > representation of the materialized state of the events, so
> that
> > > > for the
> > > > > > > > same segment only the latest state is stored. Besides
> reducing
> > > > storage
> > > > > > > > footprint, this also is likely to relate directly with the
> > > > in-memory
> > > > > > > > representation of the RLMM cache (which probably is some
> kind of
> > > a
> > > > Map
> > > > > > with
> > > > > > > > key being segment ID and value being the segment state), so
> > > > recovery
> > > > > > from
> > > > > > > > disk will be straightforward.
> > > > > > > >
> > > > > > > > 9008. Under "Topic deletion lifecycle", step (1), it will be
> > > > useful to
> > > > > > > > mention when in the deletion flow does the controller
> publish the
> > > > > > > > delete_partition_marked event to say that the partition is
> marked
> > > > for
> > > > > > > > deletion?
> > > > > > > >
> > > > > > > > 9009. There are ~4 TODOs in the KIP. Could you please address
> > > > these or
> > > > > > > > remove them?
> > > > > > > >
> > > > > > > > 9010. There is a reference to a Google doc on the KIP which
> was
> > > > used
> > > > > > > > earlier for discussions. Please could you remove the
> reference,
> > > > since
> > > > > > the
> > > > > > > > KIP is the source of the truth?
> > > > > > > >
> > > > > > > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > > > > > > >
> > > > > > > >
> > > > > > > > Cheers,
> > > > > > > > Kowshik
> > > > > > > >
> > > > > > > >
> > > > > > > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> > > > > > satish.duggana@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Jun,
> > > > > > > > > Thanks for your comments. Please go through the inline
> replies.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 5102.2: It seems that both positions can just be int.
> Another
> > > > option
> > > > > > is
> > > > > > > > to
> > > > > > > > > have two methods. Would it be clearer?
> > > > > > > > >
> > > > > > > > >     InputStream
> fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > remoteLogSegmentMetadata,  int startPosition)
> > > > > > > > throwsRemoteStorageException;
> > > > > > > > >
> > > > > > > > >     InputStream
> fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > remoteLogSegmentMetadata, int startPosition, int
> endPosition)
> > > > throws
> > > > > > > > > RemoteStorageException;
> > > > > > > > >
> > > > > > > > > That makes sense to me, updated the KIP.
> > > > > > > > >
> > > > > > > > > 6003: Could you also update the javadoc for the return
> value?
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > > log.retention.bytes
> > > > > > > > > to be consistent with local.log.retention.ms?
> > > > > > > > >
> > > > > > > > > Yes, it can be defaulted to log.retention.bytes.
> > > > > > > > >
> > > > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > > > >
> > > > > > > > > Added TopicIdPartition in the KIP.
> > > > > > > > >
> > > > > > > > > 6022: For all public facing classes, could you specify the
> > > > package
> > > > > > name?
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Satish.
> > > > > > > > >
> > > > > > > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > > >
> > > > > > > > > > Hi, Satish,
> > > > > > > > > >
> > > > > > > > > > Thanks for the reply. A few more comments below.
> > > > > > > > > >
> > > > > > > > > > 5102.2: It seems that both positions can just be int.
> Another
> > > > > > option is
> > > > > > > > > to
> > > > > > > > > > have two methods. Would it be clearer?
> > > > > > > > > >
> > > > > > > > > >     InputStream
> fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > remoteLogSegmentMetadata,
> > > > > > > > > >                                     int startPosition)
> throws
> > > > > > > > > > RemoteStorageException;
> > > > > > > > > >
> > > > > > > > > >     InputStream
> fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > remoteLogSegmentMetadata,
> > > > > > > > > >                                     int startPosition,
> int
> > > > > > endPosition)
> > > > > > > > > > throws RemoteStorageException;
> > > > > > > > > >
> > > > > > > > > > 6003: Could you also update the javadoc for the return
> value?
> > > > > > > > > >
> > > > > > > > > > 6010: What kind of tiering throughput have you seen with
> 5
> > > > threads?
> > > > > > > > > >
> > > > > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > > > > log.retention.bytes
> > > > > > > > > > to be consistent with local.log.retention.ms?
> > > > > > > > > >
> > > > > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > > > > >
> > > > > > > > > > 6022: For all public facing classes, could you specify
> the
> > > > package
> > > > > > > > name?
> > > > > > > > > >
> > > > > > > > > > It seems that you already added the topicId support. Two
> > > other
> > > > > > > > remaining
> > > > > > > > > > items are (a) the format of local tier metadata storage
> and
> > > (b)
> > > > > > > > upgrade.
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Jun,
> > > > > > > > > > > Thanks for your comments. Please find the inline
> replies
> > > > below.
> > > > > > > > > > >
> > > > > > > > > > > >605.2 It's rare for the follower to need the remote
> data.
> > > > So,
> > > > > > the
> > > > > > > > > current
> > > > > > > > > > > approach is fine too. Could you document the process of
> > > > > > rebuilding
> > > > > > > > the
> > > > > > > > > > > producer state since we can't simply trim the
> producerState
> > > > to an
> > > > > > > > > offset in
> > > > > > > > > > > the middle of a segment.
> > > > > > > > > > >
> > > > > > > > > > > Will clarify in the KIP.
> > > > > > > > > > >
> > > > > > > > > > > >5102.2 Would it be clearer to make startPosiont long
> and
> > > > > > endPosition
> > > > > > > > > of
> > > > > > > > > > > Optional<Long>?
> > > > > > > > > > >
> > > > > > > > > > > We will have arg checks with respective validation. It
> is
> > > > not a
> > > > > > good
> > > > > > > > > > > practice to have arguments with optional as mentioned
> here.
> > > > > > > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as
> File
> > > > > > instead of
> > > > > > > > > > > ByteBuffer.
> > > > > > > > > > >
> > > > > > > > > > > Updated.
> > > > > > > > > > >
> > > > > > > > > > > >5102.7 Could you define all public methods for
> > > > LogSegmentData?
> > > > > > > > > > >
> > > > > > > > > > > Updated.
> > > > > > > > > > >
> > > > > > > > > > > >5103.5 Could you change the reference to
> > > > > > rlm_process_interval_ms and
> > > > > > > > > > > rlm_retry_interval_ms to the new config names? Also,
> the
> > > > retry
> > > > > > > > interval
> > > > > > > > > > > config seems still missing. It would be useful to
> support
> > > > > > exponential
> > > > > > > > > > > backoff with the retry interval config.
> > > > > > > > > > >
> > > > > > > > > > > Good point. We wanted the retry with truncated
> exponential
> > > > > > backoff,
> > > > > > > > > > > updated the KIP.
> > > > > > > > > > >
> > > > > > > > > > > >5111. "RLM follower fetches the earliest offset for
> the
> > > > earliest
> > > > > > > > > leader
> > > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > > topicPartition,
> > > > > > > > > int
> > > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> > > This
> > > > > > text is
> > > > > > > > > still
> > > > > > > > > > > there. Also, could we remove earliestLogOffset() from
> RLMM?
> > > > > > > > > > >
> > > > > > > > > > > Updated.
> > > > > > > > > > >
> > > > > > > > > > > >5115. There are still references to "remote log
> cleaners".
> > > > > > > > > > >
> > > > > > > > > > > Updated.
> > > > > > > > > > >
> > > > > > > > > > > >6000. Since we are returning new error codes, we need
> to
> > > > bump
> > > > > > up the
> > > > > > > > > > > protocol version for Fetch request. Also, it will be
> useful
> > > > to
> > > > > > > > > document all
> > > > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > > > >
> > > > > > > > > > > Sure, we will add that in the KIP.
> > > > > > > > > > >
> > > > > > > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs():
> > > > Currently,
> > > > > > > > > leaderEpoch
> > > > > > > > > > > is int32 instead of long.
> > > > > > > > > > >
> > > > > > > > > > > Updated.
> > > > > > > > > > >
> > > > > > > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> > > needed
> > > > > > given
> > > > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > > > >
> > > > > > > > > > > No, it is fixed.
> > > > > > > > > > >
> > > > > > > > > > > >6003. RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > > topicPartition, long offset, int epochForOffset):
> Should
> > > this
> > > > > > return
> > > > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > > > >
> > > > > > > > > > > That makes sense, updated.
> > > > > > > > > > >
> > > > > > > > > > > >6005. RemoteLogState: It seems it's better to split it
> > > > between
> > > > > > > > > > > DeletePartitionUpdate and
> RemoteLogSegmentMetadataUpdate
> > > > since
> > > > > > the
> > > > > > > > > states
> > > > > > > > > > > are never shared between the two use cases.
> > > > > > > > > > >
> > > > > > > > > > > Agree with that, updated.
> > > > > > > > > > >
> > > > > > > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may
> be ok.
> > > > > > However,
> > > > > > > > > is it
> > > > > > > > > > > ture that other than the metadata topic, RLMM just
> needs to
> > > > know
> > > > > > > > > whether
> > > > > > > > > > > there is a replica assigned to this broker and doesn't
> need
> > > > to
> > > > > > know
> > > > > > > > > whether
> > > > > > > > > > > the replica is the leader or the follower?
> > > > > > > > > > >
> > > > > > > > > > > That may be true. If the implementation does not need
> that,
> > > > it
> > > > > > can
> > > > > > > > > > > ignore the information in the callback.
> > > > > > > > > > >
> > > > > > > > > > > >6007: "Handle expired remote segments (leader and
> > > > follower)":
> > > > > > Why is
> > > > > > > > > this
> > > > > > > > > > > needed in both the leader and the follower?
> > > > > > > > > > >
> > > > > > > > > > > Updated.
> > > > > > > > > > >
> > > > > > > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > > > > > > >                 "type": "int64",
> > > > > > > > > > > The segment size can just be int32.
> > > > > > > > > > >
> > > > > > > > > > > Updated.
> > > > > > > > > > >
> > > > > > > > > > > >6009. For the record format in the log, it seems that
> we
> > > > need
> > > > > > to add
> > > > > > > > > > > record
> > > > > > > > > > > type and record version before the serialized bytes.
> We can
> > > > > > follow
> > > > > > > > the
> > > > > > > > > > > convention used in
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > > >
> > > > > > > > > > > Yes, KIP already mentions that these are serialized
> before
> > > > the
> > > > > > > > payload
> > > > > > > > > > > as below. We will mention explicitly that these two are
> > > > written
> > > > > > > > before
> > > > > > > > > > > the data is written.
> > > > > > > > > > >
> > > > > > > > > > > RLMM instance on broker publishes the message to the
> topic
> > > > with
> > > > > > key
> > > > > > > > as
> > > > > > > > > > > null and value with the below format.
> > > > > > > > > > >
> > > > > > > > > > > type      : unsigned var int, represents the value
> type.
> > > This
> > > > > > value
> > > > > > > > is
> > > > > > > > > > > 'apikey' as mentioned in the schema.
> > > > > > > > > > > version : unsigned var int, the 'version' number of the
> > > type
> > > > as
> > > > > > > > > > > mentioned in the schema.
> > > > > > > > > > > data      : record payload in kafka protocol message
> > > format.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > >6010. remote.log.manager.thread.pool.size: The default
> > > > value is
> > > > > > 10.
> > > > > > > > > This
> > > > > > > > > > > might be too high when enabling the tiered feature for
> the
> > > > first
> > > > > > > > time.
> > > > > > > > > > > Since there are lots of segments that need to be tiered
> > > > > > initially, a
> > > > > > > > > large
> > > > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > > > >
> > > > > > > > > > > Is the default value 5 reasonable?
> > > > > > > > > > >
> > > > > > > > > > > 6011. "The number of milli seconds to keep the local
> log
> > > > segment
> > > > > > > > > before it
> > > > > > > > > > > gets deleted. If not set, the value in
> > > > `log.retention.minutes` is
> > > > > > > > > used. If
> > > > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > > log.retention.ms
> > > > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > > > Nice typo catch. Updated the KIP.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Satish.
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <
> jun@confluent.io>
> > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> below.
> > > > > > > > > > > >
> > > > > > > > > > > > 605.2 It's rare for the follower to need the remote
> data.
> > > > So,
> > > > > > the
> > > > > > > > > current
> > > > > > > > > > > > approach is fine too. Could you document the process
> of
> > > > > > rebuilding
> > > > > > > > > the
> > > > > > > > > > > > producer state since we can't simply trim the
> > > > producerState to
> > > > > > an
> > > > > > > > > offset
> > > > > > > > > > > in
> > > > > > > > > > > > the middle of a segment.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.2 Would it be clearer to make startPosiont long
> and
> > > > > > > > endPosition
> > > > > > > > > of
> > > > > > > > > > > > Optional<Long>?
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as
> File
> > > > > > instead of
> > > > > > > > > > > > ByteBuffer.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.7 Could you define all public methods for
> > > > LogSegmentData?
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.5 Could you change the reference to
> > > > > > rlm_process_interval_ms
> > > > > > > > and
> > > > > > > > > > > > rlm_retry_interval_ms to the new config names? Also,
> the
> > > > retry
> > > > > > > > > interval
> > > > > > > > > > > > config seems still missing. It would be useful to
> support
> > > > > > > > exponential
> > > > > > > > > > > > backoff with the retry interval config.
> > > > > > > > > > > >
> > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for
> the
> > > > > > earliest
> > > > > > > > > leader
> > > > > > > > > > > > epoch by calling
> RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > topicPartition,
> > > > > > > > > > > int
> > > > > > > > > > > > leaderEpoch) and updates that as the log start
> offset."
> > > > This
> > > > > > text
> > > > > > > > is
> > > > > > > > > > > still
> > > > > > > > > > > > there. Also, could we remove earliestLogOffset() from
> > > RLMM?
> > > > > > > > > > > >
> > > > > > > > > > > > 5115. There are still references to "remote log
> > > cleaners".
> > > > > > > > > > > >
> > > > > > > > > > > > 6000. Since we are returning new error codes, we
> need to
> > > > bump
> > > > > > up
> > > > > > > > the
> > > > > > > > > > > > protocol version for Fetch request. Also, it will be
> > > > useful to
> > > > > > > > > document
> > > > > > > > > > > all
> > > > > > > > > > > > new error codes and whether they are retriable or
> not.
> > > > > > > > > > > >
> > > > > > > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs():
> > > > Currently,
> > > > > > > > > > > leaderEpoch
> > > > > > > > > > > > is int32 instead of long.
> > > > > > > > > > > >
> > > > > > > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> > > > needed
> > > > > > given
> > > > > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > > > > >
> > > > > > > > > > > > 6003. RemoteLogSegmentMetadata
> > > > > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > > > topicPartition, long offset, int epochForOffset):
> Should
> > > > this
> > > > > > > > return
> > > > > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > > > > >
> > > > > > > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be
> useful
> > > to
> > > > > > pick a
> > > > > > > > > more
> > > > > > > > > > > > indicative name so that people understand what epoch
> this
> > > > is.
> > > > > > > > > > > >
> > > > > > > > > > > > 6005. RemoteLogState: It seems it's better to split
> it
> > > > between
> > > > > > > > > > > > DeletePartitionUpdate and
> RemoteLogSegmentMetadataUpdate
> > > > since
> > > > > > the
> > > > > > > > > states
> > > > > > > > > > > > are never shared between the two use cases.
> > > > > > > > > > > >
> > > > > > > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may
> be
> > > ok.
> > > > > > However,
> > > > > > > > > is it
> > > > > > > > > > > > ture that other than the metadata topic, RLMM just
> needs
> > > to
> > > > > > know
> > > > > > > > > whether
> > > > > > > > > > > > there is a replica assigned to this broker and
> doesn't
> > > > need to
> > > > > > know
> > > > > > > > > > > whether
> > > > > > > > > > > > the replica is the leader or the follower?
> > > > > > > > > > > >
> > > > > > > > > > > > 6007: "Handle expired remote segments (leader and
> > > > follower)":
> > > > > > Why
> > > > > > > > is
> > > > > > > > > this
> > > > > > > > > > > > needed in both the leader and the follower?
> > > > > > > > > > > >
> > > > > > > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > > > > > > >                 "type": "int64",
> > > > > > > > > > > > The segment size can just be int32.
> > > > > > > > > > > >
> > > > > > > > > > > > 6009. For the record format in the log, it seems
> that we
> > > > need
> > > > > > to
> > > > > > > > add
> > > > > > > > > > > record
> > > > > > > > > > > > type and record version before the serialized bytes.
> We
> > > can
> > > > > > follow
> > > > > > > > > the
> > > > > > > > > > > > convention used in
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > > > > .
> > > > > > > > > > > >
> > > > > > > > > > > > 6010. remote.log.manager.thread.pool.size: The
> default
> > > > value
> > > > > > is 10.
> > > > > > > > > This
> > > > > > > > > > > > might be too high when enabling the tiered feature
> for
> > > the
> > > > > > first
> > > > > > > > > time.
> > > > > > > > > > > > Since there are lots of segments that need to be
> tiered
> > > > > > initially,
> > > > > > > > a
> > > > > > > > > > > large
> > > > > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > > > > >
> > > > > > > > > > > > 6011. "The number of milli seconds to keep the local
> log
> > > > > > segment
> > > > > > > > > before
> > > > > > > > > > > it
> > > > > > > > > > > > gets deleted. If not set, the value in
> > > > `log.retention.minutes`
> > > > > > is
> > > > > > > > > used.
> > > > > > > > > > > If
> > > > > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > > > > log.retention.ms
> > > > > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi,
> > > > > > > > > > > > > We updated the KIP with the points mentioned in the
> > > > earlier
> > > > > > mail
> > > > > > > > > > > > > except for KIP-516 related changes. You can go
> through
> > > > them
> > > > > > and
> > > > > > > > > let us
> > > > > > > > > > > > > know if you have any comments. We will update the
> KIP
> > > > with
> > > > > > the
> > > > > > > > > > > > > remaining todo items and KIP-516 related changes
> by end
> > > > of
> > > > > > this
> > > > > > > > > > > > > week(5th Dec).
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Satish.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > Thanks for your comments. Please find the inline
> > > > replies
> > > > > > below.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 605.2 "Build the local leader epoch cache by
> cutting
> > > > the
> > > > > > leader
> > > > > > > > > epoch
> > > > > > > > > > > > > > sequence received from remote storage to [LSO,
> > > ELO]." I
> > > > > > > > > mentioned an
> > > > > > > > > > > > > issue
> > > > > > > > > > > > > > earlier. Suppose the leader's local start offset
> is
> > > > 100.
> > > > > > The
> > > > > > > > > follower
> > > > > > > > > > > > > finds
> > > > > > > > > > > > > > a remote segment covering offset range [80,
> 120). The
> > > > > > > > > producerState
> > > > > > > > > > > with
> > > > > > > > > > > > > > this remote segment is up to offset 120. To trim
> the
> > > > > > > > > producerState to
> > > > > > > > > > > > > > offset 100 requires more work since one needs to
> > > > download
> > > > > > the
> > > > > > > > > > > previous
> > > > > > > > > > > > > > producerState up to offset 80 and then replay the
> > > > messages
> > > > > > from
> > > > > > > > > 80 to
> > > > > > > > > > > > > 100.
> > > > > > > > > > > > > > It seems that it's simpler in this case for the
> > > > follower
> > > > > > just
> > > > > > > > to
> > > > > > > > > > > take the
> > > > > > > > > > > > > > remote segment as it is and start fetching from
> > > offset
> > > > 120.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We chose that approach to avoid any edge cases
> here.
> > > It
> > > > > > may be
> > > > > > > > > > > > > > possible that the remote log segment that is
> received
> > > > may
> > > > > > not
> > > > > > > > > have
> > > > > > > > > > > the
> > > > > > > > > > > > > > same leader epoch sequence from 100-120 as it
> > > contains
> > > > on
> > > > > > the
> > > > > > > > > > > > > > leader(this can happen due to unclean leader).
> It is
> > > > safe
> > > > > > to
> > > > > > > > > start
> > > > > > > > > > > > > > from what the leader returns here.Another way is
> to
> > > > find
> > > > > > the
> > > > > > > > > remote
> > > > > > > > > > > > > > log segment
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It
> seems
> > > > that
> > > > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only
> called on
> > > > the
> > > > > > > > > replicas
> > > > > > > > > > > for a
> > > > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > obtains the metadata for remote segments for
> > > deletion.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > RLMM will always receive the callback for the
> remote
> > > > log
> > > > > > > > metadata
> > > > > > > > > > > > > > topic partitions hosted on the local broker and
> these
> > > > will
> > > > > > be
> > > > > > > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > > > implemented
> > > > > > now.
> > > > > > > > > Could
> > > > > > > > > > > you
> > > > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We mentioned KIP-516 and how it helps. We will
> update
> > > > this
> > > > > > KIP
> > > > > > > > > with
> > > > > > > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > > > > following two
> > > > > > > > > APIs
> > > > > > > > > > > are
> > > > > > > > > > > > > > used. According to the wiki, the former is used
> for
> > > > topic
> > > > > > > > > deletion
> > > > > > > > > > > and
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > latter is used for retention. It seems that
> retention
> > > > > > should
> > > > > > > > use
> > > > > > > > > the
> > > > > > > > > > > > > former
> > > > > > > > > > > > > > since remote segments without a matching epoch
> in the
> > > > > > leader
> > > > > > > > > > > (potentially
> > > > > > > > > > > > > > due to unclean leader election) also need to be
> > > garbage
> > > > > > > > > collected.
> > > > > > > > > > > The
> > > > > > > > > > > > > > latter seems to be used for the new leader to
> > > > determine the
> > > > > > > > last
> > > > > > > > > > > tiered
> > > > > > > > > > > > > > segment.
> > > > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> topicPartition)
> > > > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Right,.that is what we are currently doing. We
> will
> > > > update
> > > > > > the
> > > > > > > > > > > > > > javadocs and wiki with that. Earlier, we did not
> want
> > > > to
> > > > > > remove
> > > > > > > > > the
> > > > > > > > > > > > > > segments which are not matched with leader epochs
> > > from
> > > > the
> > > > > > > > ladder
> > > > > > > > > > > > > > partition as they may be used later by a replica
> > > which
> > > > can
> > > > > > > > > become a
> > > > > > > > > > > > > > leader (unclean leader election) and refer those
> > > > segments.
> > > > > > But
> > > > > > > > > that
> > > > > > > > > > > > > > may leak these segments in remote storage until
> the
> > > > topic
> > > > > > > > > lifetime.
> > > > > > > > > > > We
> > > > > > > > > > > > > > decided to cleanup the segments with the oldest
> > > incase
> > > > of
> > > > > > size
> > > > > > > > > based
> > > > > > > > > > > > > > retention also.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it
> > > seems
> > > > > > that
> > > > > > > > > they can
> > > > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > It will be useful to have metadata for RSM to
> fetch
> > > log
> > > > > > > > segment.
> > > > > > > > > It
> > > > > > > > > > > > > > may create location/path using id with other
> metadata
> > > > too.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use
> long
> > > > > > instead of
> > > > > > > > > Long?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Wanted to keep endPosition as optional to read
> till
> > > the
> > > > > > end of
> > > > > > > > > the
> > > > > > > > > > > > > > segment and avoid sentinels.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > > implementation
> > > > > > > > > and
> > > > > > > > > > > > > others
> > > > > > > > > > > > > > Don't?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Actually,  RSM will not have any default
> > > > implementations.
> > > > > > > > Those 3
> > > > > > > > > > > > > > methods were made default earlier for tests etc.
> > > > Updated
> > > > > > the
> > > > > > > > > wiki.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102.4. Could we define
> > > RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Sure, they will be added.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier
> to
> > > > pass
> > > > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array
> > > than
> > > > a
> > > > > > file
> > > > > > > > > since
> > > > > > > > > > > it
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Right, this is in plan.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it
> > > needs
> > > > > > both
> > > > > > > > > > > baseOffset
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > startOffset. For example, deleteRecords() could
> move
> > > > the
> > > > > > > > > startOffset
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > middle of a segment. If we copy the full segment
> to
> > > > remote
> > > > > > > > > storage,
> > > > > > > > > > > the
> > > > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Good point. startOffset is baseOffset by
> default, if
> > > > not
> > > > > > set
> > > > > > > > > > > explicitly.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Sure, updated the wiki.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > > > 5103.1 Could we define the default value of
> > > > non-required
> > > > > > > > configs
> > > > > > > > > > > (e.g the
> > > > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Sure, that makes sense.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms
> should
> > > > > > default to
> > > > > > > > > > > > > retention.ms,
> > > > > > > > > > > > > > instead of remote.log.retention.minutes.
> Similarly,
> > > it
> > > > > > seems
> > > > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > > > segment.bytes.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Right, we do not have  remote.log.retention as we
> > > > discussed
> > > > > > > > > earlier.
> > > > > > > > > > > > > > Thanks for catching the typo.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > > description
> > > > > > > > says
> > > > > > > > > > > "used in
> > > > > > > > > > > > > > scheduling tasks to copy segments, fetch remote
> log
> > > > > > indexes and
> > > > > > > > > > > clean up
> > > > > > > > > > > > > > remote log segments". However, there is a
> separate
> > > > > > > > > > > > > > config remote.log.reader.threads for fetching
> remote
> > > > data.
> > > > > > It's
> > > > > > > > > > > weird to
> > > > > > > > > > > > > > fetch remote index and log in different thread
> pools
> > > > since
> > > > > > both
> > > > > > > > > are
> > > > > > > > > > > used
> > > > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Right, remote.log.manager.thread.pool is mainly
> used
> > > > for
> > > > > > > > > copy/cleanup
> > > > > > > > > > > > > > activities. Fetch path always goes through
> > > > > > > > > remote.log.reader.threads.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is
> that
> > > > the
> > > > > > amount
> > > > > > > > > of
> > > > > > > > > > > time
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > back off when there is no work to do? If so,
> perhaps
> > > it
> > > > > > can be
> > > > > > > > > > > renamed as
> > > > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > This is the delay interval for each iteration.
> It may
> > > > be
> > > > > > > > renamed
> > > > > > > > > to
> > > > > > > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > > > rlm_retry_interval_ms
> > > > > > > > > > > configs? If
> > > > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > remote.log.manager.task.interval.ms is the
> process
> > > > > > internal,
> > > > > > > > > retry
> > > > > > > > > > > > > > interval is missing in the configs, which will be
> > > > updated
> > > > > > in
> > > > > > > > the
> > > > > > > > > KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly
> LRU) of
> > > > the
> > > > > > index
> > > > > > > > > > > files of
> > > > > > > > > > > > > > remote log segments to avoid multiple index
> fetches
> > > > from
> > > > > > the
> > > > > > > > > remote
> > > > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on
> > > disk,
> > > > > > where is
> > > > > > > > > it
> > > > > > > > > > > > > stored?
> > > > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > It is stored on disk. They are stored in a
> directory
> > > > > > > > > > > > > > `remote-log-index-cache` under log dir. We plan
> to
> > > > have a
> > > > > > > > config
> > > > > > > > > for
> > > > > > > > > > > > > > that instead of default. We will have a
> configuration
> > > > for
> > > > > > that.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> > > Earliest
> > > > > > Local
> > > > > > > > > Offset
> > > > > > > > > > > in
> > > > > > > > > > > > > > different places. It would be useful to
> standardize
> > > the
> > > > > > > > > terminology.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Sure.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux
> state".
> > > > It
> > > > > > listed
> > > > > > > > > two
> > > > > > > > > > > > > options
> > > > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > > > We already mentioned in the KIP that we chose
> > > option-2.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5107. Follower to leader transition: It has step
> 2,
> > > > but not
> > > > > > > > step
> > > > > > > > > 1.
> > > > > > > > > > > > > > Step-1 is there but it is not explicitly
> highlighted.
> > > > It is
> > > > > > > > > previous
> > > > > > > > > > > > > > table to step-2.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5108. If a consumer fetches from the remote data
> and
> > > > the
> > > > > > remote
> > > > > > > > > > > storage
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > not available, what error code is used in the
> fetch
> > > > > > response?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Good point. We have not yet defined the error for
> > > this
> > > > > > case. We
> > > > > > > > > need
> > > > > > > > > > > > > > to define an error message and send the same in
> fetch
> > > > > > response.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it
> returns
> > > the
> > > > > > first
> > > > > > > > > message
> > > > > > > > > > > > > > offset whose timestamp is >= to the given
> timestamp
> > > in
> > > > the
> > > > > > > > > request.
> > > > > > > > > > > That
> > > > > > > > > > > > > > means it checks in remote log time indexes first,
> > > after
> > > > > > which
> > > > > > > > > local
> > > > > > > > > > > log
> > > > > > > > > > > > > > time indexes are checked." Could you document
> which
> > > > method
> > > > > > in
> > > > > > > > > RLMM is
> > > > > > > > > > > > > used
> > > > > > > > > > > > > > for this?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Okay.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> > > segment
> > > > > > metadata
> > > > > > > > > of
> > > > > > > > > > > that
> > > > > > > > > > > > > > partition with a delete marker and publishes
> them to
> > > > RLMM."
> > > > > > > > This
> > > > > > > > > > > seems
> > > > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset
> for
> > > the
> > > > > > > > earliest
> > > > > > > > > > > leader
> > > > > > > > > > > > > > epoch by calling
> > > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > int
> > > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > > > offset." Do
> > > > > > we
> > > > > > > > > need
> > > > > > > > > > > that
> > > > > > > > > > > > > > since replication propagates logStartOffset
> already?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Good point. Right, existing replication protocol
> > > takes
> > > > > > care of
> > > > > > > > > > > > > > updating the followers’s log start offset
> received
> > > > from the
> > > > > > > > > leader.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough
> for
> > > > fetching
> > > > > > > > from
> > > > > > > > > > > remote
> > > > > > > > > > > > > > storage?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Remote reads may fail within the current default
> wait
> > > > > > time, but
> > > > > > > > > > > > > > subsequent fetches would be able to serve as that
> > > data
> > > > is
> > > > > > > > stored
> > > > > > > > > in
> > > > > > > > > > > > > > the local cache. This cache is currently
> implemented
> > > in
> > > > > > RSMs.
> > > > > > > > > But we
> > > > > > > > > > > > > > plan to pull this into the remote log messaging
> layer
> > > > in
> > > > > > > > future.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5113. "Committed offsets can be stored in a local
> > > file
> > > > to
> > > > > > avoid
> > > > > > > > > > > reading
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > messages again when a broker is restarted."
> Could you
> > > > > > describe
> > > > > > > > > the
> > > > > > > > > > > format
> > > > > > > > > > > > > > and the location of the file? Also, could the
> same
> > > > message
> > > > > > be
> > > > > > > > > > > processed
> > > > > > > > > > > > > by
> > > > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> > > > handle
> > > > > > that?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > > > 5114.1 There are two records named
> > > > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > > > with
> > > > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > > > document
> > > > > > > > whether
> > > > > > > > > > > > > endOffset
> > > > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> > > > explain
> > > > > > > > > LeaderEpoch
> > > > > > > > > > > a
> > > > > > > > > > > > > bit
> > > > > > > > > > > > > > more? Is that the epoch of the leader when it
> copies
> > > > the
> > > > > > > > segment
> > > > > > > > > to
> > > > > > > > > > > > > remote
> > > > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Right, this is the leader epoch of the broker
> which
> > > > copied
> > > > > > this
> > > > > > > > > > > > > > segment. This is helpful in reason about which
> broker
> > > > > > copied
> > > > > > > > the
> > > > > > > > > > > > > > segment to remote storage.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a
> bit
> > > > more?
> > > > > > Each
> > > > > > > > > > > record in
> > > > > > > > > > > > > > Kafka already has a timestamp field. Could we
> just
> > > use
> > > > > > that?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > This is the  timestamp at which  the respective
> event
> > > > > > occurred.
> > > > > > > > > Added
> > > > > > > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can
> be  any
> > > > other
> > > > > > > > > > > > > > implementation. We thought about that but it
> looked
> > > > > > cleaner to
> > > > > > > > > use at
> > > > > > > > > > > > > > the message structure level instead of getting
> that
> > > > from
> > > > > > the
> > > > > > > > > consumer
> > > > > > > > > > > > > > record and using that to build the respective
> event.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be
> int32?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Right, it looks like config allows only int
> value >=
> > > > 14.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be
> confused
> > > > with
> > > > > > the
> > > > > > > > log
> > > > > > > > > > > cleaner
> > > > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth
> like
> > > > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I am fine with RemotePartitionRemover or
> > > > > > > > > RemoteLogDeletionManager(we
> > > > > > > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked
> and
> > > > > > processes
> > > > > > > > it
> > > > > > > > > if
> > > > > > > > > > > it is
> > > > > > > > > > > > > > not yet processed earlier." How does it know
> whether
> > > > > > > > > > > > > > delete_partition_marked has been processed
> earlier?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > This is to handle duplicate
> delete_partition_marked
> > > > > > events. RLC
> > > > > > > > > > > > > > internally maintains a state for the
> delete_partition
> > > > > > events
> > > > > > > > and
> > > > > > > > > if
> > > > > > > > > > > it
> > > > > > > > > > > > > > already has an existing event then it ignores if
> it
> > > is
> > > > > > already
> > > > > > > > > being
> > > > > > > > > > > > > > processed.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5117. Should we add a new MessageFormatter to
> read
> > > the
> > > > tier
> > > > > > > > > metadata
> > > > > > > > > > > > > topic?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Right, this is in plan but did not mention it in
> the
> > > > KIP.
> > > > > > This
> > > > > > > > > will
> > > > > > > > > > > be
> > > > > > > > > > > > > > useful for debugging purposes too.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task
> > > queue
> > > > > > size.
> > > > > > > > If
> > > > > > > > > the
> > > > > > > > > > > task
> > > > > > > > > > > > > > queue is full, broker will stop reading remote
> log
> > > > > > segments."
> > > > > > > > > What
> > > > > > > > > > > do we
> > > > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We return an error response for that partition.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5119. It would be useful to list all things not
> > > > supported
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > first
> > > > > > > > > > > > > > version in a Future work or Limitations section.
> For
> > > > > > example,
> > > > > > > > > > > compacted
> > > > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable
> from
> > > > true
> > > > > > to
> > > > > > > > > false,
> > > > > > > > > > > etc.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We already have a non-goals section which is
> filled
> > > > with
> > > > > > some
> > > > > > > > of
> > > > > > > > > > > these
> > > > > > > > > > > > > > details. Do we need another limitations section?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <
> > > > jun@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > > > below.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 605.2 "Build the local leader epoch cache by
> > > cutting
> > > > the
> > > > > > > > leader
> > > > > > > > > > > epoch
> > > > > > > > > > > > > > > sequence received from remote storage to [LSO,
> > > > ELO]." I
> > > > > > > > > mentioned
> > > > > > > > > > > an
> > > > > > > > > > > > > issue
> > > > > > > > > > > > > > > earlier. Suppose the leader's local start
> offset is
> > > > 100.
> > > > > > The
> > > > > > > > > > > follower
> > > > > > > > > > > > > finds
> > > > > > > > > > > > > > > a remote segment covering offset range [80,
> 120).
> > > The
> > > > > > > > > producerState
> > > > > > > > > > > > > with
> > > > > > > > > > > > > > > this remote segment is up to offset 120. To
> trim
> > > the
> > > > > > > > > producerState
> > > > > > > > > > > to
> > > > > > > > > > > > > > > offset 100 requires more work since one needs
> to
> > > > > > download the
> > > > > > > > > > > previous
> > > > > > > > > > > > > > > producerState up to offset 80 and then replay
> the
> > > > > > messages
> > > > > > > > > from 80
> > > > > > > > > > > to
> > > > > > > > > > > > > 100.
> > > > > > > > > > > > > > > It seems that it's simpler in this case for the
> > > > follower
> > > > > > just
> > > > > > > > > to
> > > > > > > > > > > take
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > remote segment as it is and start fetching from
> > > > offset
> > > > > > 120.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It
> > > seems
> > > > that
> > > > > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only
> called
> > > > on the
> > > > > > > > > replicas
> > > > > > > > > > > for
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > > obtains the metadata for remote segments for
> > > > deletion.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > > > implemented
> > > > > > now.
> > > > > > > > > > > Could you
> > > > > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how
> the
> > > > > > following
> > > > > > > > two
> > > > > > > > > > > APIs
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > used. According to the wiki, the former is
> used for
> > > > topic
> > > > > > > > > deletion
> > > > > > > > > > > and
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > latter is used for retention. It seems that
> > > retention
> > > > > > should
> > > > > > > > > use
> > > > > > > > > > > the
> > > > > > > > > > > > > former
> > > > > > > > > > > > > > > since remote segments without a matching epoch
> in
> > > the
> > > > > > leader
> > > > > > > > > > > > > (potentially
> > > > > > > > > > > > > > > due to unclean leader election) also need to be
> > > > garbage
> > > > > > > > > collected.
> > > > > > > > > > > The
> > > > > > > > > > > > > > > latter seems to be used for the new leader to
> > > > determine
> > > > > > the
> > > > > > > > > last
> > > > > > > > > > > tiered
> > > > > > > > > > > > > > > segment.
> > > > > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > topicPartition)
> > > > > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(),
> it
> > > > seems
> > > > > > that
> > > > > > > > > they
> > > > > > > > > > > can
> > > > > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > > > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use
> long
> > > > > > instead
> > > > > > > > of
> > > > > > > > > > > Long?
> > > > > > > > > > > > > > > 5102.3 Why only some of the methods have
> default
> > > > > > > > > implementation and
> > > > > > > > > > > > > others
> > > > > > > > > > > > > > > don't?
> > > > > > > > > > > > > > > 5102.4. Could we define
> > > > RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's
> easier to
> > > > pass
> > > > > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte
> array
> > > > than a
> > > > > > file
> > > > > > > > > > > since it
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that
> it
> > > > needs
> > > > > > both
> > > > > > > > > > > > > baseOffset and
> > > > > > > > > > > > > > > startOffset. For example, deleteRecords() could
> > > move
> > > > the
> > > > > > > > > > > startOffset
> > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > middle of a segment. If we copy the full
> segment to
> > > > > > remote
> > > > > > > > > > > storage, the
> > > > > > > > > > > > > > > baseOffset and the startOffset will be
> different.
> > > > > > > > > > > > > > > 5102.7 Could we define all the public methods
> for
> > > > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > > > > 5103.1 Could we define the default value of
> > > > non-required
> > > > > > > > > configs
> > > > > > > > > > > (e.g
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms
> should
> > > > > > default
> > > > > > > > to
> > > > > > > > > > > > > retention.ms,
> > > > > > > > > > > > > > > instead of remote.log.retention.minutes.
> Similarly,
> > > > it
> > > > > > seems
> > > > > > > > > > > > > > > that local.log.retention.bytes should default
> to
> > > > > > > > segment.bytes.
> > > > > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > > > > description
> > > > > > > > > says
> > > > > > > > > > > "used
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > scheduling tasks to copy segments, fetch
> remote log
> > > > > > indexes
> > > > > > > > and
> > > > > > > > > > > clean
> > > > > > > > > > > > > up
> > > > > > > > > > > > > > > remote log segments". However, there is a
> separate
> > > > > > > > > > > > > > > config remote.log.reader.threads for fetching
> > > remote
> > > > > > data.
> > > > > > > > It's
> > > > > > > > > > > weird
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > fetch remote index and log in different thread
> > > pools
> > > > > > since
> > > > > > > > > both are
> > > > > > > > > > > > > used
> > > > > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is
> > > that
> > > > the
> > > > > > > > > amount of
> > > > > > > > > > > > > time to
> > > > > > > > > > > > > > > back off when there is no work to do? If so,
> > > perhaps
> > > > it
> > > > > > can
> > > > > > > > be
> > > > > > > > > > > renamed
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > > > rlm_retry_interval_ms
> > > > > > > > > > > configs?
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly
> LRU)
> > > > of the
> > > > > > > > index
> > > > > > > > > > > files
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > remote log segments to avoid multiple index
> fetches
> > > > from
> > > > > > the
> > > > > > > > > remote
> > > > > > > > > > > > > > > storage." Is the RLM in memory or on disk? If
> on
> > > > disk,
> > > > > > where
> > > > > > > > > is it
> > > > > > > > > > > > > stored?
> > > > > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> > > > Earliest
> > > > > > Local
> > > > > > > > > > > Offset in
> > > > > > > > > > > > > > > different places. It would be useful to
> standardize
> > > > the
> > > > > > > > > > > terminology.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux
> > > > state". It
> > > > > > > > > listed two
> > > > > > > > > > > > > options
> > > > > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5107. Follower to leader transition: It has
> step 2,
> > > > but
> > > > > > not
> > > > > > > > > step 1.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5108. If a consumer fetches from the remote
> data
> > > and
> > > > the
> > > > > > > > remote
> > > > > > > > > > > > > storage is
> > > > > > > > > > > > > > > not available, what error code is used in the
> fetch
> > > > > > response?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it
> returns
> > > > the
> > > > > > first
> > > > > > > > > > > message
> > > > > > > > > > > > > > > offset whose timestamp is >= to the given
> timestamp
> > > > in
> > > > > > the
> > > > > > > > > request.
> > > > > > > > > > > > > That
> > > > > > > > > > > > > > > means it checks in remote log time indexes
> first,
> > > > after
> > > > > > which
> > > > > > > > > > > local log
> > > > > > > > > > > > > > > time indexes are checked." Could you document
> which
> > > > > > method in
> > > > > > > > > RLMM
> > > > > > > > > > > is
> > > > > > > > > > > > > used
> > > > > > > > > > > > > > > for this?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> > > > segment
> > > > > > > > > metadata of
> > > > > > > > > > > that
> > > > > > > > > > > > > > > partition with a delete marker and publishes
> them
> > > to
> > > > > > RLMM."
> > > > > > > > > This
> > > > > > > > > > > seems
> > > > > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5111. "RLM follower fetches the earliest
> offset for
> > > > the
> > > > > > > > > earliest
> > > > > > > > > > > leader
> > > > > > > > > > > > > > > epoch by calling
> > > > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > int
> > > > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > > > offset."
> > > > > > Do we
> > > > > > > > > need
> > > > > > > > > > > that
> > > > > > > > > > > > > > > since replication propagates logStartOffset
> > > already?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough
> for
> > > > > > fetching
> > > > > > > > > from
> > > > > > > > > > > remote
> > > > > > > > > > > > > > > storage?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5113. "Committed offsets can be stored in a
> local
> > > > file to
> > > > > > > > avoid
> > > > > > > > > > > > > reading the
> > > > > > > > > > > > > > > messages again when a broker is restarted."
> Could
> > > you
> > > > > > > > describe
> > > > > > > > > the
> > > > > > > > > > > > > format
> > > > > > > > > > > > > > > and the location of the file? Also, could the
> same
> > > > > > message be
> > > > > > > > > > > > > processed by
> > > > > > > > > > > > > > > RLMM again after broker restart? If so, how do
> we
> > > > handle
> > > > > > > > that?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > > > > 5114.1 There are two records named
> > > > > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > > > with
> > > > > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > > > document
> > > > > > > > > whether
> > > > > > > > > > > > > endOffset
> > > > > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could
> you
> > > > explain
> > > > > > > > > > > LeaderEpoch a
> > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > more? Is that the epoch of the leader when it
> > > copies
> > > > the
> > > > > > > > > segment to
> > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this
> a bit
> > > > more?
> > > > > > > > Each
> > > > > > > > > > > record
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > Kafka already has a timestamp field. Could we
> just
> > > > use
> > > > > > that?
> > > > > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be
> > > int32?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be
> confused
> > > > with
> > > > > > the
> > > > > > > > > log
> > > > > > > > > > > > > cleaner
> > > > > > > > > > > > > > > for compaction. Perhaps it can be renamed to
> sth
> > > like
> > > > > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5116. "RLC receives the
> delete_partition_marked and
> > > > > > processes
> > > > > > > > > it
> > > > > > > > > > > if it
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > not yet processed earlier." How does it know
> > > whether
> > > > > > > > > > > > > > > delete_partition_marked has been processed
> earlier?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5117. Should we add a new MessageFormatter to
> read
> > > > the
> > > > > > tier
> > > > > > > > > > > metadata
> > > > > > > > > > > > > topic?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5118. "Maximum remote log reader thread pool
> task
> > > > queue
> > > > > > size.
> > > > > > > > > If
> > > > > > > > > > > the
> > > > > > > > > > > > > task
> > > > > > > > > > > > > > > queue is full, broker will stop reading remote
> log
> > > > > > segments."
> > > > > > > > > What
> > > > > > > > > > > do
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5119. It would be useful to list all things not
> > > > > > supported in
> > > > > > > > > the
> > > > > > > > > > > first
> > > > > > > > > > > > > > > version in a Future work or Limitations
> section.
> > > For
> > > > > > example,
> > > > > > > > > > > compacted
> > > > > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable
> > > from
> > > > > > true to
> > > > > > > > > false,
> > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik
> Prakasam <
> > > > > > > > > > > > > kprakasam@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.duggana@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.duggana@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
> > > <
> > > > > > > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > > > > > > (
> > > > > > > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > > > > > > kprakasam@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.ch@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 (
> ch@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.duggana@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
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> > >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Manikumar,
Thanks for your comment. Please find the inline replies below.

9201. Can we make it clear by mentioning RLM as an internal component and
RLMM and RSM are pluggable components.
It's also good to update/add the RLM, RLMM components  diagrams with
internal components/tasks.

Updated the KIP.

9202. RLMM metadata topic is a non-compact topic. Can we make clear that
metadata topic retention time should be greater
 than user topics. or How about setting retention set to -1 (unlimited) and
let users to modify it.

I am fine with -1 and updated the configuration doc and made it clear
in the KIP.

9203. RLMM has additional responsibility of handling topic delete requests.
With current design, RLMM implementations need
to implement deletion logic. How about making RemotePartitionRemover as a
separate task, which can be reused..

Good point. We had plans to address that in a future KIP, added in
future work items.

9204. Can we list all the new error codes added by the KIP?

Those are mentioned in the protocol changes section.

9205. Can we also support  `remote.log.metadata.producr.*`,
`remote.log.metadata.consumer.*` prefixed configs to
override any default configs ?. also include RLMM cache configs, if any.

Good point. Updated the KIP.

9206. In the Upgrade section, we have below statement
"Before enabling tiered storage, you should make sure the producer
snapshots are built for all the segments for that topic
in all followers. You should wait till the log retention occurs for all the
segments so that all the segments have producer snapshots."
Can you add more details about this constraint?

Sure, updated in the KIP.


~Satish.

On Tue, 2 Feb 2021 at 21:04, Manikumar <ma...@gmail.com> wrote:
>
> Hi Satish,
>
> Thanks for the KIP.  some of my comments below.
>
> 9201. Can we make it clear by mentioning RLM as an internal component and
> RLMM and RSM are pluggable components.
> It's also good to update/add the RLM, RLMM components  diagrams with
> internal components/tasks.
>
> 9202. RLMM metadata topic is a non-compact topic. Can we make clear that
> metadata topic retention time should be greater
>  than user topics. or How about setting retention set to -1 (unlimited) and
> let users to modify it.
>
> 9203. RLMM has additional responsibility of handling topic delete requests.
> With current design, RLMM implementations need
> to implement deletion logic. How about making RemotePartitionRemover as a
> separate task, which can be reused..
>
> 9204. Can we list all the new error codes added by the KIP?
>
> 9205. Can we also support  `remote.log.metadata.producr.*`,
> `remote.log.metadata.consumer.*` prefixed configs to
> override any default configs ?. also include RLMM cache configs, if any.
>
> 9206. In the Upgrade section, we have below statement
> "Before enabling tiered storage, you should make sure the producer
> snapshots are built for all the segments for that topic
> in all followers. You should wait till the log retention occurs for all the
> segments so that all the segments have producer snapshots."
>
>  Can you add more details about this constraint?
>
>
> Thanks,
> Manikumar
>
> On Tue, Jan 26, 2021 at 1:12 AM Jun Rao <ju...@confluent.io> wrote:
>
> > Hi, Satish,
> >
> > Thanks for the reply. A few more comments below.
> >
> > 6000. When RSM/RLMM is not available, it seems that we need to return a
> > retriable error to the affected client requests. So, should we add a new
> > error code for requests like fetch and listOffset?
> >
> > 9111. RemotePartitionRemover:
> > 91111.1 Could we make it clear that it's part of the default RLMM
> > implementation and a separate implementation is needed if one has a
> > customized RLMM?
> > 91111.2 "RPRM gets all the remote log segments for the partition using RLMM
> > and each of these remote log segments is deleted with the next steps.RLMM
> > subscribes to the local remote log metadata partitions and it will have the
> > segment metadata of all the user topic partitions associated with that
> > remote log metadata partition." It seems that RLMM needs to subscribe to
> > the remote log metadata partitions first before those segments can be
> > deleted?
> > 9111.3 There are still references to "remote log cleaners". They need to be
> > replaced with RemotePartitionRemover.
> >
> > 9114.1 Could we add the requirement on log.message.format before enabling
> > tiered storage?
> >
> > 9116. RemoteLogMetadataFormatter: This is used with the ConsoleConsumer
> > tool, right? Then, are those new options for ConsoleConsumer and how are
> > they passed in?
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Mon, Jan 25, 2021 at 8:02 AM Satish Duggana <sa...@gmail.com>
> > wrote:
> >
> > > Hi Jun,
> > >
> > > Thanks for your comments. Please find the inline replies below.
> > > Several of these were discussed/clarified in our last discussion.
> > >
> > > 6000. Since we are returning new error codes, we need to bump up the
> > > protocol version for Fetch request. Also, it will be useful to document
> > all
> > > new error codes and whether they are retriable or not.
> > > This previous comment doesn't seem to be addressed.
> > >
> > > Updated KIP on fetch protocol and error code.
> > >
> > > 9110. flat_file_format: Could you define the serialized representation
> > for
> > > each field in the header?
> > >
> > > Added the serialized representation types in the KIP.
> > >
> > > 9111. RPM has the following 2 steps.
> > > "1. The controller publishes delete_partition_marked event to say that
> > the
> > > partition is marked for deletion. There can be multiple events published
> > > when the controller restarts or failover and this event will be
> > > deduplicated by RPM.
> > > 2. RPM receives the delete_partition_marked and processes it if it is not
> > > yet processed earlier."
> > > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> > default
> > > RLMM implementation or is it meant for any RLMM implementation?
> > >
> > > RPM(or RPRM) is only for the default RLMM implementation. RPM receives
> > > the delete_partition_marked events from RLMM and acts on them. Updated
> > > KIP with more details.
> > >
> > > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to
> > make
> > > it clear in the comment that this is for the initial retry backoff.
> > >
> > > Updated the KIP.
> > >
> > > 9113. RLMM:
> > > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> > updated.
> > >
> > > Updated javadoc of this method.
> > >
> > > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> > > since all other update events have leaderEpoch?
> > >
> > > +1 to have that,  updated the KIP.
> > >
> > > 9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
> > > to make it clear this is for deletion?
> > >
> > > Sure, updated the KIP.
> > >
> > > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
> > > clear this is for segment?
> > >
> > > Sure, Updated the KIP.
> > >
> > > 9114.Upgrade:
> > > 9114.1 It seems that we require message format > 0.11 for turning on the
> > > remote store feature.
> > >
> > > 9114.2 It's not clear to me why remote.log.storage.system.enable needs to
> > > be set to true before bumping up inter.broker.protocol.version.
> > >
> > > Agree that based on our discussion, this is not required. Upgrade
> > > notes is updated.
> > >
> > > 9114.3 "If the topic-id is not received in the LeaderAndIsr request then
> > > remote log storage will not start. But it will log an error message in
> > the
> > > log. One way to address this is to do a rolling restart of that broker,
> > so
> > > that the leader will be moved to another broker and the controller will
> > > send LeaderAndIsr with the registered topic-id." Why is this needed? With
> > > KIP-516, after upgrading to the latest protocol, topicIds are auto
> > created.
> > >
> > > I thought there may be edge cases of not receiving topic-id but you
> > > clarified in the meeting that won’t be the case. I agree that it is
> > > not needed.
> > >
> > > 9115. testing: Could you comment on how we plan to do integration and
> > > system tests? Do we plan to include, for example, an in-memory
> > > implementation of RSM?
> > >
> > > For integration tests, we can have file based RSM that we have. For
> > > system tests, we can have a single node HDFS cluster in one of the
> > > containers and use HDFS RSM implementation.
> > >
> > > 9116. There is still a todo for the message formatter.
> > >
> > > Updated the KIP with the format.
> > >
> > >
> > > Satish.
> > >
> > >
> > > On Tue, 12 Jan 2021 at 07:07, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > Hi, Satish,
> > > >
> > > > Thanks for the reply. A few more followup comments.
> > > >
> > > > 6000. Since we are returning new error codes, we need to bump up the
> > > > protocol version for Fetch request. Also, it will be useful to document
> > > all
> > > > new error codes and whether they are retriable or not.
> > > > This previous comment doesn't seem to be addressed.
> > > >
> > > > 9110. flat_file_format: Could you define the serialized representation
> > > for
> > > > each field in the header?
> > > >
> > > > 9111. RPM has the following 2 steps.
> > > > "1. The controller publishes delete_partition_marked event to say that
> > > the
> > > > partition is marked for deletion. There can be multiple events
> > published
> > > > when the controller restarts or failover and this event will be
> > > > deduplicated by RPM.
> > > > 2. RPM receives the delete_partition_marked and processes it if it is
> > not
> > > > yet processed earlier."
> > > > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> > > default
> > > > RLMM implementation or is it meant for any RLMM implementation?
> > > >
> > > > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to
> > > make
> > > > it clear in the comment that this is for the initial retry backoff.
> > > >
> > > > 9113. RLMM:
> > > > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> > > updated.
> > > > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> > > > since all other update events have leaderEpoch?
> > > > 9113.3 Could we rename RemotePartitionState to
> > RemotePartitionDeleteState
> > > > to make it clear this is for deletion?
> > > > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make
> > it
> > > > clear this is for segment?
> > > >
> > > > 9114.Upgrade:
> > > > 9114.1 It seems that we require message format > 0.11 for turning on
> > the
> > > > remote store feature.
> > > > 9114.2 It's not clear to me why remote.log.storage.system.enable needs
> > to
> > > > be set to true before bumping up inter.broker.protocol.version.
> > > > 9114.3 "If the topic-id is not received in the LeaderAndIsr request
> > then
> > > > remote log storage will not start. But it will log an error message in
> > > the
> > > > log. One way to address this is to do a rolling restart of that broker,
> > > so
> > > > that the leader will be moved to another broker and the controller will
> > > > send LeaderAndIsr with the registered topic-id." Why is this needed?
> > With
> > > > KIP-516, after upgrading to the latest protocol, topicIds are auto
> > > created.
> > > >
> > > > 9115. testing: Could you comment on how we plan to do integration and
> > > > system tests? Do we plan to include, for example, an in-memory
> > > > implementation of RSM?
> > > >
> > > > 9116. There is still a todo for the message formatter.
> > > >
> > > > Jun
> > > >
> > > > On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <
> > satish.duggana@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Jun,
> > > > > Thanks for your comments. Please find the inline replies below.
> > > > >
> > > > > 6022. For packages used for server plugins, the convention is to
> > > > > use org.apache.kafka.server. See java-based Authorizer as an example.
> > > > >
> > > > > Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> > > > > ‘org.apache.kafka.server.log.remote.storage’.  Updated in the KIP.
> > > > >
> > > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The
> > > flat
> > > > > file captures the state of the remote segments. After a partition is
> > > > > deleted, it seems that we just need to remove the partitions's remote
> > > > > segments from the flat file.
> > > > >
> > > > > DeletePartitionState might not yet have been processed by RPM and not
> > > > > completed. We  will not have  that in flat  file format  once it
> > > > > reaches DELETE_PARTITION_FINISHED state.
> > > > >
> > > > > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > > > > version. It seems that's doable. One can just do the normal upgrade
> > > first
> > > > > and wait enough time (for producer snapshots to be built), and then
> > > enable
> > > > > remote storage.
> > > > >
> > > > > Upgrade notes updates in the KIP.
> > > > >
> > > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > > > > tracking the remote segments when RLMM.onPartitionLeadershipChanges()
> > > is
> > > > > called with the broker being the leader for __remote_log_metadata
> > > > > partition? If so, could we document it?
> > > > >
> > > > > The current plan is to have that as part of RLMM and RPM uses that to
> > > > > get the remote segments list. I will add this detail in the
> > respective
> > > > > sections.
> > > > >
> > > > > Satish.
> > > > >
> > > > > On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > Hi, Satish,
> > > > > >
> > > > > > Thanks for the reply. A few more followup comments.
> > > > > >
> > > > > > 6022. For packages used for server plugins, the convention is to
> > > > > > use org.apache.kafka.server. See java-based Authorizer as an
> > example.
> > > > > >
> > > > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format?
> > The
> > > flat
> > > > > > file captures the state of the remote segments. After a partition
> > is
> > > > > > deleted, it seems that we just need to remove the partitions's
> > remote
> > > > > > segments from the flat file.
> > > > > >
> > > > > > 9101. Upgrade: It will be useful to allow direct upgrade from an
> > old
> > > > > > version. It seems that's doable. One can just do the normal upgrade
> > > first
> > > > > > and wait enough time (for producer snapshots to be built), and then
> > > > > enable
> > > > > > remote storage.
> > > > > >
> > > > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM
> > starts
> > > > > > tracking the remote segments when
> > > RLMM.onPartitionLeadershipChanges() is
> > > > > > called with the broker being the leader for __remote_log_metadata
> > > > > > partition? If so, could we document it?
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <
> > > kprakasam@confluent.io
> > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Satish,
> > > > > > >
> > > > > > > Thanks for the updates! A few more comments below.
> > > > > > >
> > > > > > > 9001. Under the "Upgrade" section, there is a line mentioning:
> > > > > "Upgrade the
> > > > > > > existing Kafka cluster to 2.7 version and allow this to run for
> > > the log
> > > > > > > retention of user topics that you want to enable tiered storage.
> > > This
> > > > > will
> > > > > > > allow all the topics to have the producer snapshots generated for
> > > each
> > > > > log
> > > > > > > segment." -- Which associated change in AK were you referring to
> > > here?
> > > > > Is
> > > > > > > it: https://github.com/apache/kafka/pull/7929 ? It seems like I
> > > don't
> > > > > see
> > > > > > > it in the 2.7 release branch yet, here is the link:
> > > > > > > https://github.com/apache/kafka/commits/2.7.
> > > > > > >
> > > > > > > 9002. Under the "Upgrade" section, the configuration mentioned is
> > > > > > > 'remote.log.storage.system.enable'. However, under "Public
> > > Interfaces"
> > > > > > > section the corresponding configuration is
> > > > > 'remote.storage.system.enable'.
> > > > > > > Could we use the same one in both, maybe
> > > > > > > 'remote.log.storage.system.enable'?
> > > > > > >
> > > > > > > 9003. Under "Per Topic Configuration", the KIP recommends setting
> > > > > > > 'remote.log.storage.enable' to true at a per-topic level. It will
> > > be
> > > > > useful
> > > > > > > to add a line that if the user wants to enable it for all topics,
> > > then
> > > > > they
> > > > > > > should be able to set the cluster-wide default to true. Also, it
> > > will
> > > > > be
> > > > > > > useful to mention that the KIP currently does not support setting
> > > it to
> > > > > > > false (after it is set to true), and add that to the future work
> > > > > section.
> > > > > > >
> > > > > > > 9004. Under "Committed offsets file format", the sample provided
> > > shows
> > > > > > > partition number and offset. Is the topic name required for
> > > identifying
> > > > > > > which topic the partitions belong to?
> > > > > > >
> > > > > > > 9005. Under "Internal flat-file store format of remote log
> > > metadata",
> > > > > it
> > > > > > > seems useful to specify both topic name and topic ID for
> > debugging
> > > > > > > purposes.
> > > > > > >
> > > > > > > 9006. Under "Internal flat-file store format of remote log
> > > metadata",
> > > > > the
> > > > > > > description of "metadata-topic-offset" currently says "offset of
> > > the
> > > > > remote
> > > > > > > log metadata topic from which this topic partition's remote log
> > > > > metadata is
> > > > > > > fetched." Just for the wording, perhaps you meant to refer to the
> > > > > offset
> > > > > > > upto which the file has been committed? i.e. "offset of the
> > remote
> > > log
> > > > > > > metadata topic upto which this topic partition's remote log
> > > metadata
> > > > > has
> > > > > > > been committed into this file."
> > > > > > >
> > > > > > > 9007. Under "Internal flat-file store format of remote log
> > > metadata",
> > > > > the
> > > > > > > schema of the payload (i.e. beyond the header) seems to contain
> > the
> > > > > events
> > > > > > > from the metadata topic. It seems useful to instead persist the
> > > > > > > representation of the materialized state of the events, so that
> > > for the
> > > > > > > same segment only the latest state is stored. Besides reducing
> > > storage
> > > > > > > footprint, this also is likely to relate directly with the
> > > in-memory
> > > > > > > representation of the RLMM cache (which probably is some kind of
> > a
> > > Map
> > > > > with
> > > > > > > key being segment ID and value being the segment state), so
> > > recovery
> > > > > from
> > > > > > > disk will be straightforward.
> > > > > > >
> > > > > > > 9008. Under "Topic deletion lifecycle", step (1), it will be
> > > useful to
> > > > > > > mention when in the deletion flow does the controller publish the
> > > > > > > delete_partition_marked event to say that the partition is marked
> > > for
> > > > > > > deletion?
> > > > > > >
> > > > > > > 9009. There are ~4 TODOs in the KIP. Could you please address
> > > these or
> > > > > > > remove them?
> > > > > > >
> > > > > > > 9010. There is a reference to a Google doc on the KIP which was
> > > used
> > > > > > > earlier for discussions. Please could you remove the reference,
> > > since
> > > > > the
> > > > > > > KIP is the source of the truth?
> > > > > > >
> > > > > > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > > > > > >
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Kowshik
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> > > > > satish.duggana@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Jun,
> > > > > > > > Thanks for your comments. Please go through the inline replies.
> > > > > > > >
> > > > > > > >
> > > > > > > > 5102.2: It seems that both positions can just be int. Another
> > > option
> > > > > is
> > > > > > > to
> > > > > > > > have two methods. Would it be clearer?
> > > > > > > >
> > > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata,  int startPosition)
> > > > > > > throwsRemoteStorageException;
> > > > > > > >
> > > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata, int startPosition, int endPosition)
> > > throws
> > > > > > > > RemoteStorageException;
> > > > > > > >
> > > > > > > > That makes sense to me, updated the KIP.
> > > > > > > >
> > > > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > log.retention.bytes
> > > > > > > > to be consistent with local.log.retention.ms?
> > > > > > > >
> > > > > > > > Yes, it can be defaulted to log.retention.bytes.
> > > > > > > >
> > > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > > >
> > > > > > > > Added TopicIdPartition in the KIP.
> > > > > > > >
> > > > > > > > 6022: For all public facing classes, could you specify the
> > > package
> > > > > name?
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Satish.
> > > > > > > >
> > > > > > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > > >
> > > > > > > > > Hi, Satish,
> > > > > > > > >
> > > > > > > > > Thanks for the reply. A few more comments below.
> > > > > > > > >
> > > > > > > > > 5102.2: It seems that both positions can just be int. Another
> > > > > option is
> > > > > > > > to
> > > > > > > > > have two methods. Would it be clearer?
> > > > > > > > >
> > > > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > remoteLogSegmentMetadata,
> > > > > > > > >                                     int startPosition) throws
> > > > > > > > > RemoteStorageException;
> > > > > > > > >
> > > > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > remoteLogSegmentMetadata,
> > > > > > > > >                                     int startPosition, int
> > > > > endPosition)
> > > > > > > > > throws RemoteStorageException;
> > > > > > > > >
> > > > > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > > > > >
> > > > > > > > > 6010: What kind of tiering throughput have you seen with 5
> > > threads?
> > > > > > > > >
> > > > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > > > log.retention.bytes
> > > > > > > > > to be consistent with local.log.retention.ms?
> > > > > > > > >
> > > > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > > > >
> > > > > > > > > 6022: For all public facing classes, could you specify the
> > > package
> > > > > > > name?
> > > > > > > > >
> > > > > > > > > It seems that you already added the topicId support. Two
> > other
> > > > > > > remaining
> > > > > > > > > items are (a) the format of local tier metadata storage and
> > (b)
> > > > > > > upgrade.
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > > > > > satish.duggana@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Jun,
> > > > > > > > > > Thanks for your comments. Please find the inline replies
> > > below.
> > > > > > > > > >
> > > > > > > > > > >605.2 It's rare for the follower to need the remote data.
> > > So,
> > > > > the
> > > > > > > > current
> > > > > > > > > > approach is fine too. Could you document the process of
> > > > > rebuilding
> > > > > > > the
> > > > > > > > > > producer state since we can't simply trim the producerState
> > > to an
> > > > > > > > offset in
> > > > > > > > > > the middle of a segment.
> > > > > > > > > >
> > > > > > > > > > Will clarify in the KIP.
> > > > > > > > > >
> > > > > > > > > > >5102.2 Would it be clearer to make startPosiont long and
> > > > > endPosition
> > > > > > > > of
> > > > > > > > > > Optional<Long>?
> > > > > > > > > >
> > > > > > > > > > We will have arg checks with respective validation. It is
> > > not a
> > > > > good
> > > > > > > > > > practice to have arguments with optional as mentioned here.
> > > > > > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as File
> > > > > instead of
> > > > > > > > > > ByteBuffer.
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > > >5102.7 Could you define all public methods for
> > > LogSegmentData?
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > > >5103.5 Could you change the reference to
> > > > > rlm_process_interval_ms and
> > > > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> > > retry
> > > > > > > interval
> > > > > > > > > > config seems still missing. It would be useful to support
> > > > > exponential
> > > > > > > > > > backoff with the retry interval config.
> > > > > > > > > >
> > > > > > > > > > Good point. We wanted the retry with truncated exponential
> > > > > backoff,
> > > > > > > > > > updated the KIP.
> > > > > > > > > >
> > > > > > > > > > >5111. "RLM follower fetches the earliest offset for the
> > > earliest
> > > > > > > > leader
> > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > int
> > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> > This
> > > > > text is
> > > > > > > > still
> > > > > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > > >5115. There are still references to "remote log cleaners".
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > > >6000. Since we are returning new error codes, we need to
> > > bump
> > > > > up the
> > > > > > > > > > protocol version for Fetch request. Also, it will be useful
> > > to
> > > > > > > > document all
> > > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > > >
> > > > > > > > > > Sure, we will add that in the KIP.
> > > > > > > > > >
> > > > > > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs():
> > > Currently,
> > > > > > > > leaderEpoch
> > > > > > > > > > is int32 instead of long.
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> > needed
> > > > > given
> > > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > > >
> > > > > > > > > > No, it is fixed.
> > > > > > > > > >
> > > > > > > > > > >6003. RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > topicPartition, long offset, int epochForOffset): Should
> > this
> > > > > return
> > > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > > >
> > > > > > > > > > That makes sense, updated.
> > > > > > > > > >
> > > > > > > > > > >6005. RemoteLogState: It seems it's better to split it
> > > between
> > > > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> > > since
> > > > > the
> > > > > > > > states
> > > > > > > > > > are never shared between the two use cases.
> > > > > > > > > >
> > > > > > > > > > Agree with that, updated.
> > > > > > > > > >
> > > > > > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > > > > However,
> > > > > > > > is it
> > > > > > > > > > ture that other than the metadata topic, RLMM just needs to
> > > know
> > > > > > > > whether
> > > > > > > > > > there is a replica assigned to this broker and doesn't need
> > > to
> > > > > know
> > > > > > > > whether
> > > > > > > > > > the replica is the leader or the follower?
> > > > > > > > > >
> > > > > > > > > > That may be true. If the implementation does not need that,
> > > it
> > > > > can
> > > > > > > > > > ignore the information in the callback.
> > > > > > > > > >
> > > > > > > > > > >6007: "Handle expired remote segments (leader and
> > > follower)":
> > > > > Why is
> > > > > > > > this
> > > > > > > > > > needed in both the leader and the follower?
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > > > > > >                 "type": "int64",
> > > > > > > > > > The segment size can just be int32.
> > > > > > > > > >
> > > > > > > > > > Updated.
> > > > > > > > > >
> > > > > > > > > > >6009. For the record format in the log, it seems that we
> > > need
> > > > > to add
> > > > > > > > > > record
> > > > > > > > > > type and record version before the serialized bytes. We can
> > > > > follow
> > > > > > > the
> > > > > > > > > > convention used in
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > >
> > > > > > > > > > Yes, KIP already mentions that these are serialized before
> > > the
> > > > > > > payload
> > > > > > > > > > as below. We will mention explicitly that these two are
> > > written
> > > > > > > before
> > > > > > > > > > the data is written.
> > > > > > > > > >
> > > > > > > > > > RLMM instance on broker publishes the message to the topic
> > > with
> > > > > key
> > > > > > > as
> > > > > > > > > > null and value with the below format.
> > > > > > > > > >
> > > > > > > > > > type      : unsigned var int, represents the value type.
> > This
> > > > > value
> > > > > > > is
> > > > > > > > > > 'apikey' as mentioned in the schema.
> > > > > > > > > > version : unsigned var int, the 'version' number of the
> > type
> > > as
> > > > > > > > > > mentioned in the schema.
> > > > > > > > > > data      : record payload in kafka protocol message
> > format.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > >6010. remote.log.manager.thread.pool.size: The default
> > > value is
> > > > > 10.
> > > > > > > > This
> > > > > > > > > > might be too high when enabling the tiered feature for the
> > > first
> > > > > > > time.
> > > > > > > > > > Since there are lots of segments that need to be tiered
> > > > > initially, a
> > > > > > > > large
> > > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > > >
> > > > > > > > > > Is the default value 5 reasonable?
> > > > > > > > > >
> > > > > > > > > > 6011. "The number of milli seconds to keep the local log
> > > segment
> > > > > > > > before it
> > > > > > > > > > gets deleted. If not set, the value in
> > > `log.retention.minutes` is
> > > > > > > > used. If
> > > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > log.retention.ms
> > > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > > Nice typo catch. Updated the KIP.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Satish.
> > > > > > > > > >
> > > > > > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > Hi, Satish,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > > > >
> > > > > > > > > > > 605.2 It's rare for the follower to need the remote data.
> > > So,
> > > > > the
> > > > > > > > current
> > > > > > > > > > > approach is fine too. Could you document the process of
> > > > > rebuilding
> > > > > > > > the
> > > > > > > > > > > producer state since we can't simply trim the
> > > producerState to
> > > > > an
> > > > > > > > offset
> > > > > > > > > > in
> > > > > > > > > > > the middle of a segment.
> > > > > > > > > > >
> > > > > > > > > > > 5102.2 Would it be clearer to make startPosiont long and
> > > > > > > endPosition
> > > > > > > > of
> > > > > > > > > > > Optional<Long>?
> > > > > > > > > > >
> > > > > > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File
> > > > > instead of
> > > > > > > > > > > ByteBuffer.
> > > > > > > > > > >
> > > > > > > > > > > 5102.7 Could you define all public methods for
> > > LogSegmentData?
> > > > > > > > > > >
> > > > > > > > > > > 5103.5 Could you change the reference to
> > > > > rlm_process_interval_ms
> > > > > > > and
> > > > > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> > > retry
> > > > > > > > interval
> > > > > > > > > > > config seems still missing. It would be useful to support
> > > > > > > exponential
> > > > > > > > > > > backoff with the retry interval config.
> > > > > > > > > > >
> > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > > earliest
> > > > > > > > leader
> > > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > > topicPartition,
> > > > > > > > > > int
> > > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> > > This
> > > > > text
> > > > > > > is
> > > > > > > > > > still
> > > > > > > > > > > there. Also, could we remove earliestLogOffset() from
> > RLMM?
> > > > > > > > > > >
> > > > > > > > > > > 5115. There are still references to "remote log
> > cleaners".
> > > > > > > > > > >
> > > > > > > > > > > 6000. Since we are returning new error codes, we need to
> > > bump
> > > > > up
> > > > > > > the
> > > > > > > > > > > protocol version for Fetch request. Also, it will be
> > > useful to
> > > > > > > > document
> > > > > > > > > > all
> > > > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > > > >
> > > > > > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs():
> > > Currently,
> > > > > > > > > > leaderEpoch
> > > > > > > > > > > is int32 instead of long.
> > > > > > > > > > >
> > > > > > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> > > needed
> > > > > given
> > > > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > > > >
> > > > > > > > > > > 6003. RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > > topicPartition, long offset, int epochForOffset): Should
> > > this
> > > > > > > return
> > > > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > > > >
> > > > > > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful
> > to
> > > > > pick a
> > > > > > > > more
> > > > > > > > > > > indicative name so that people understand what epoch this
> > > is.
> > > > > > > > > > >
> > > > > > > > > > > 6005. RemoteLogState: It seems it's better to split it
> > > between
> > > > > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> > > since
> > > > > the
> > > > > > > > states
> > > > > > > > > > > are never shared between the two use cases.
> > > > > > > > > > >
> > > > > > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be
> > ok.
> > > > > However,
> > > > > > > > is it
> > > > > > > > > > > ture that other than the metadata topic, RLMM just needs
> > to
> > > > > know
> > > > > > > > whether
> > > > > > > > > > > there is a replica assigned to this broker and doesn't
> > > need to
> > > > > know
> > > > > > > > > > whether
> > > > > > > > > > > the replica is the leader or the follower?
> > > > > > > > > > >
> > > > > > > > > > > 6007: "Handle expired remote segments (leader and
> > > follower)":
> > > > > Why
> > > > > > > is
> > > > > > > > this
> > > > > > > > > > > needed in both the leader and the follower?
> > > > > > > > > > >
> > > > > > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > > > > > >                 "type": "int64",
> > > > > > > > > > > The segment size can just be int32.
> > > > > > > > > > >
> > > > > > > > > > > 6009. For the record format in the log, it seems that we
> > > need
> > > > > to
> > > > > > > add
> > > > > > > > > > record
> > > > > > > > > > > type and record version before the serialized bytes. We
> > can
> > > > > follow
> > > > > > > > the
> > > > > > > > > > > convention used in
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > > > .
> > > > > > > > > > >
> > > > > > > > > > > 6010. remote.log.manager.thread.pool.size: The default
> > > value
> > > > > is 10.
> > > > > > > > This
> > > > > > > > > > > might be too high when enabling the tiered feature for
> > the
> > > > > first
> > > > > > > > time.
> > > > > > > > > > > Since there are lots of segments that need to be tiered
> > > > > initially,
> > > > > > > a
> > > > > > > > > > large
> > > > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > > > >
> > > > > > > > > > > 6011. "The number of milli seconds to keep the local log
> > > > > segment
> > > > > > > > before
> > > > > > > > > > it
> > > > > > > > > > > gets deleted. If not set, the value in
> > > `log.retention.minutes`
> > > > > is
> > > > > > > > used.
> > > > > > > > > > If
> > > > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > > > log.retention.ms
> > > > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi,
> > > > > > > > > > > > We updated the KIP with the points mentioned in the
> > > earlier
> > > > > mail
> > > > > > > > > > > > except for KIP-516 related changes. You can go through
> > > them
> > > > > and
> > > > > > > > let us
> > > > > > > > > > > > know if you have any comments. We will update the KIP
> > > with
> > > > > the
> > > > > > > > > > > > remaining todo items and KIP-516 related changes by end
> > > of
> > > > > this
> > > > > > > > > > > > week(5th Dec).
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Satish.
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > Thanks for your comments. Please find the inline
> > > replies
> > > > > below.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting
> > > the
> > > > > leader
> > > > > > > > epoch
> > > > > > > > > > > > > sequence received from remote storage to [LSO,
> > ELO]." I
> > > > > > > > mentioned an
> > > > > > > > > > > > issue
> > > > > > > > > > > > > earlier. Suppose the leader's local start offset is
> > > 100.
> > > > > The
> > > > > > > > follower
> > > > > > > > > > > > finds
> > > > > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > > > > producerState
> > > > > > > > > > with
> > > > > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > > > > producerState to
> > > > > > > > > > > > > offset 100 requires more work since one needs to
> > > download
> > > > > the
> > > > > > > > > > previous
> > > > > > > > > > > > > producerState up to offset 80 and then replay the
> > > messages
> > > > > from
> > > > > > > > 80 to
> > > > > > > > > > > > 100.
> > > > > > > > > > > > > It seems that it's simpler in this case for the
> > > follower
> > > > > just
> > > > > > > to
> > > > > > > > > > take the
> > > > > > > > > > > > > remote segment as it is and start fetching from
> > offset
> > > 120.
> > > > > > > > > > > > >
> > > > > > > > > > > > > We chose that approach to avoid any edge cases here.
> > It
> > > > > may be
> > > > > > > > > > > > > possible that the remote log segment that is received
> > > may
> > > > > not
> > > > > > > > have
> > > > > > > > > > the
> > > > > > > > > > > > > same leader epoch sequence from 100-120 as it
> > contains
> > > on
> > > > > the
> > > > > > > > > > > > > leader(this can happen due to unclean leader). It is
> > > safe
> > > > > to
> > > > > > > > start
> > > > > > > > > > > > > from what the leader returns here.Another way is to
> > > find
> > > > > the
> > > > > > > > remote
> > > > > > > > > > > > > log segment
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems
> > > that
> > > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on
> > > the
> > > > > > > > replicas
> > > > > > > > > > for a
> > > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > obtains the metadata for remote segments for
> > deletion.
> > > > > > > > > > > > >
> > > > > > > > > > > > > RLMM will always receive the callback for the remote
> > > log
> > > > > > > metadata
> > > > > > > > > > > > > topic partitions hosted on the local broker and these
> > > will
> > > > > be
> > > > > > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > > implemented
> > > > > now.
> > > > > > > > Could
> > > > > > > > > > you
> > > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > > >
> > > > > > > > > > > > > We mentioned KIP-516 and how it helps. We will update
> > > this
> > > > > KIP
> > > > > > > > with
> > > > > > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > > > following two
> > > > > > > > APIs
> > > > > > > > > > are
> > > > > > > > > > > > > used. According to the wiki, the former is used for
> > > topic
> > > > > > > > deletion
> > > > > > > > > > and
> > > > > > > > > > > > the
> > > > > > > > > > > > > latter is used for retention. It seems that retention
> > > > > should
> > > > > > > use
> > > > > > > > the
> > > > > > > > > > > > former
> > > > > > > > > > > > > since remote segments without a matching epoch in the
> > > > > leader
> > > > > > > > > > (potentially
> > > > > > > > > > > > > due to unclean leader election) also need to be
> > garbage
> > > > > > > > collected.
> > > > > > > > > > The
> > > > > > > > > > > > > latter seems to be used for the new leader to
> > > determine the
> > > > > > > last
> > > > > > > > > > tiered
> > > > > > > > > > > > > segment.
> > > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > > >
> > > > > > > > > > > > > Right,.that is what we are currently doing. We will
> > > update
> > > > > the
> > > > > > > > > > > > > javadocs and wiki with that. Earlier, we did not want
> > > to
> > > > > remove
> > > > > > > > the
> > > > > > > > > > > > > segments which are not matched with leader epochs
> > from
> > > the
> > > > > > > ladder
> > > > > > > > > > > > > partition as they may be used later by a replica
> > which
> > > can
> > > > > > > > become a
> > > > > > > > > > > > > leader (unclean leader election) and refer those
> > > segments.
> > > > > But
> > > > > > > > that
> > > > > > > > > > > > > may leak these segments in remote storage until the
> > > topic
> > > > > > > > lifetime.
> > > > > > > > > > We
> > > > > > > > > > > > > decided to cleanup the segments with the oldest
> > incase
> > > of
> > > > > size
> > > > > > > > based
> > > > > > > > > > > > > retention also.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it
> > seems
> > > > > that
> > > > > > > > they can
> > > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > >
> > > > > > > > > > > > > It will be useful to have metadata for RSM to fetch
> > log
> > > > > > > segment.
> > > > > > > > It
> > > > > > > > > > > > > may create location/path using id with other metadata
> > > too.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > > > > instead of
> > > > > > > > Long?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Wanted to keep endPosition as optional to read till
> > the
> > > > > end of
> > > > > > > > the
> > > > > > > > > > > > > segment and avoid sentinels.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > implementation
> > > > > > > > and
> > > > > > > > > > > > others
> > > > > > > > > > > > > Don't?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Actually,  RSM will not have any default
> > > implementations.
> > > > > > > Those 3
> > > > > > > > > > > > > methods were made default earlier for tests etc.
> > > Updated
> > > > > the
> > > > > > > > wiki.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.4. Could we define
> > RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Sure, they will be added.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> > > pass
> > > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array
> > than
> > > a
> > > > > file
> > > > > > > > since
> > > > > > > > > > it
> > > > > > > > > > > > will
> > > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Right, this is in plan.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it
> > needs
> > > > > both
> > > > > > > > > > baseOffset
> > > > > > > > > > > > and
> > > > > > > > > > > > > startOffset. For example, deleteRecords() could move
> > > the
> > > > > > > > startOffset
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > middle of a segment. If we copy the full segment to
> > > remote
> > > > > > > > storage,
> > > > > > > > > > the
> > > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Good point. startOffset is baseOffset by default, if
> > > not
> > > > > set
> > > > > > > > > > explicitly.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Sure, updated the wiki.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > > >
> > > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > > 5103.1 Could we define the default value of
> > > non-required
> > > > > > > configs
> > > > > > > > > > (e.g the
> > > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Sure, that makes sense.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > > > > default to
> > > > > > > > > > > > retention.ms,
> > > > > > > > > > > > > instead of remote.log.retention.minutes. Similarly,
> > it
> > > > > seems
> > > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > > segment.bytes.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Right, we do not have  remote.log.retention as we
> > > discussed
> > > > > > > > earlier.
> > > > > > > > > > > > > Thanks for catching the typo.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > description
> > > > > > > says
> > > > > > > > > > "used in
> > > > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > > > > indexes and
> > > > > > > > > > clean up
> > > > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > > > config remote.log.reader.threads for fetching remote
> > > data.
> > > > > It's
> > > > > > > > > > weird to
> > > > > > > > > > > > > fetch remote index and log in different thread pools
> > > since
> > > > > both
> > > > > > > > are
> > > > > > > > > > used
> > > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Right, remote.log.manager.thread.pool is mainly used
> > > for
> > > > > > > > copy/cleanup
> > > > > > > > > > > > > activities. Fetch path always goes through
> > > > > > > > remote.log.reader.threads.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that
> > > the
> > > > > amount
> > > > > > > > of
> > > > > > > > > > time
> > > > > > > > > > > > to
> > > > > > > > > > > > > back off when there is no work to do? If so, perhaps
> > it
> > > > > can be
> > > > > > > > > > renamed as
> > > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > >
> > > > > > > > > > > > > This is the delay interval for each iteration. It may
> > > be
> > > > > > > renamed
> > > > > > > > to
> > > > > > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > > rlm_retry_interval_ms
> > > > > > > > > > configs? If
> > > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > > >
> > > > > > > > > > > > > remote.log.manager.task.interval.ms is the process
> > > > > internal,
> > > > > > > > retry
> > > > > > > > > > > > > interval is missing in the configs, which will be
> > > updated
> > > > > in
> > > > > > > the
> > > > > > > > KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of
> > > the
> > > > > index
> > > > > > > > > > files of
> > > > > > > > > > > > > remote log segments to avoid multiple index fetches
> > > from
> > > > > the
> > > > > > > > remote
> > > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on
> > disk,
> > > > > where is
> > > > > > > > it
> > > > > > > > > > > > stored?
> > > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > > >
> > > > > > > > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > > > > > > > `remote-log-index-cache` under log dir. We plan to
> > > have a
> > > > > > > config
> > > > > > > > for
> > > > > > > > > > > > > that instead of default. We will have a configuration
> > > for
> > > > > that.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> > Earliest
> > > > > Local
> > > > > > > > Offset
> > > > > > > > > > in
> > > > > > > > > > > > > different places. It would be useful to standardize
> > the
> > > > > > > > terminology.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Sure.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state".
> > > It
> > > > > listed
> > > > > > > > two
> > > > > > > > > > > > options
> > > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > > We already mentioned in the KIP that we chose
> > option-2.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5107. Follower to leader transition: It has step 2,
> > > but not
> > > > > > > step
> > > > > > > > 1.
> > > > > > > > > > > > > Step-1 is there but it is not explicitly highlighted.
> > > It is
> > > > > > > > previous
> > > > > > > > > > > > > table to step-2.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5108. If a consumer fetches from the remote data and
> > > the
> > > > > remote
> > > > > > > > > > storage
> > > > > > > > > > > > is
> > > > > > > > > > > > > not available, what error code is used in the fetch
> > > > > response?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Good point. We have not yet defined the error for
> > this
> > > > > case. We
> > > > > > > > need
> > > > > > > > > > > > > to define an error message and send the same in fetch
> > > > > response.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns
> > the
> > > > > first
> > > > > > > > message
> > > > > > > > > > > > > offset whose timestamp is >= to the given timestamp
> > in
> > > the
> > > > > > > > request.
> > > > > > > > > > That
> > > > > > > > > > > > > means it checks in remote log time indexes first,
> > after
> > > > > which
> > > > > > > > local
> > > > > > > > > > log
> > > > > > > > > > > > > time indexes are checked." Could you document which
> > > method
> > > > > in
> > > > > > > > RLMM is
> > > > > > > > > > > > used
> > > > > > > > > > > > > for this?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Okay.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> > segment
> > > > > metadata
> > > > > > > > of
> > > > > > > > > > that
> > > > > > > > > > > > > partition with a delete marker and publishes them to
> > > RLMM."
> > > > > > > This
> > > > > > > > > > seems
> > > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for
> > the
> > > > > > > earliest
> > > > > > > > > > leader
> > > > > > > > > > > > > epoch by calling
> > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > > topicPartition,
> > > > > > > > > > > > int
> > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > > offset." Do
> > > > > we
> > > > > > > > need
> > > > > > > > > > that
> > > > > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Good point. Right, existing replication protocol
> > takes
> > > > > care of
> > > > > > > > > > > > > updating the followers’s log start offset received
> > > from the
> > > > > > > > leader.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > > fetching
> > > > > > > from
> > > > > > > > > > remote
> > > > > > > > > > > > > storage?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Remote reads may fail within the current default wait
> > > > > time, but
> > > > > > > > > > > > > subsequent fetches would be able to serve as that
> > data
> > > is
> > > > > > > stored
> > > > > > > > in
> > > > > > > > > > > > > the local cache. This cache is currently implemented
> > in
> > > > > RSMs.
> > > > > > > > But we
> > > > > > > > > > > > > plan to pull this into the remote log messaging layer
> > > in
> > > > > > > future.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5113. "Committed offsets can be stored in a local
> > file
> > > to
> > > > > avoid
> > > > > > > > > > reading
> > > > > > > > > > > > the
> > > > > > > > > > > > > messages again when a broker is restarted." Could you
> > > > > describe
> > > > > > > > the
> > > > > > > > > > format
> > > > > > > > > > > > > and the location of the file? Also, could the same
> > > message
> > > > > be
> > > > > > > > > > processed
> > > > > > > > > > > > by
> > > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> > > handle
> > > > > that?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > > 5114.1 There are two records named
> > > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > > with
> > > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > > document
> > > > > > > whether
> > > > > > > > > > > > endOffset
> > > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> > > explain
> > > > > > > > LeaderEpoch
> > > > > > > > > > a
> > > > > > > > > > > > bit
> > > > > > > > > > > > > more? Is that the epoch of the leader when it copies
> > > the
> > > > > > > segment
> > > > > > > > to
> > > > > > > > > > > > remote
> > > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Right, this is the leader epoch of the broker which
> > > copied
> > > > > this
> > > > > > > > > > > > > segment. This is helpful in reason about which broker
> > > > > copied
> > > > > > > the
> > > > > > > > > > > > > segment to remote storage.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> > > more?
> > > > > Each
> > > > > > > > > > record in
> > > > > > > > > > > > > Kafka already has a timestamp field. Could we just
> > use
> > > > > that?
> > > > > > > > > > > > >
> > > > > > > > > > > > > This is the  timestamp at which  the respective event
> > > > > occurred.
> > > > > > > > Added
> > > > > > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any
> > > other
> > > > > > > > > > > > > implementation. We thought about that but it looked
> > > > > cleaner to
> > > > > > > > use at
> > > > > > > > > > > > > the message structure level instead of getting that
> > > from
> > > > > the
> > > > > > > > consumer
> > > > > > > > > > > > > record and using that to build the respective event.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Right, it looks like config allows only int value >=
> > > 14.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> > > with
> > > > > the
> > > > > > > log
> > > > > > > > > > cleaner
> > > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I am fine with RemotePartitionRemover or
> > > > > > > > RemoteLogDeletionManager(we
> > > > > > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > > > > processes
> > > > > > > it
> > > > > > > > if
> > > > > > > > > > it is
> > > > > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > > > >
> > > > > > > > > > > > > This is to handle duplicate delete_partition_marked
> > > > > events. RLC
> > > > > > > > > > > > > internally maintains a state for the delete_partition
> > > > > events
> > > > > > > and
> > > > > > > > if
> > > > > > > > > > it
> > > > > > > > > > > > > already has an existing event then it ignores if it
> > is
> > > > > already
> > > > > > > > being
> > > > > > > > > > > > > processed.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5117. Should we add a new MessageFormatter to read
> > the
> > > tier
> > > > > > > > metadata
> > > > > > > > > > > > topic?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Right, this is in plan but did not mention it in the
> > > KIP.
> > > > > This
> > > > > > > > will
> > > > > > > > > > be
> > > > > > > > > > > > > useful for debugging purposes too.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task
> > queue
> > > > > size.
> > > > > > > If
> > > > > > > > the
> > > > > > > > > > task
> > > > > > > > > > > > > queue is full, broker will stop reading remote log
> > > > > segments."
> > > > > > > > What
> > > > > > > > > > do we
> > > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > > >
> > > > > > > > > > > > > We return an error response for that partition.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5119. It would be useful to list all things not
> > > supported
> > > > > in
> > > > > > > the
> > > > > > > > > > first
> > > > > > > > > > > > > version in a Future work or Limitations section. For
> > > > > example,
> > > > > > > > > > compacted
> > > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> > > true
> > > > > to
> > > > > > > > false,
> > > > > > > > > > etc.
> > > > > > > > > > > > >
> > > > > > > > > > > > > We already have a non-goals section which is filled
> > > with
> > > > > some
> > > > > > > of
> > > > > > > > > > these
> > > > > > > > > > > > > details. Do we need another limitations section?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Satish.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <
> > > jun@confluent.io>
> > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > > below.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 605.2 "Build the local leader epoch cache by
> > cutting
> > > the
> > > > > > > leader
> > > > > > > > > > epoch
> > > > > > > > > > > > > > sequence received from remote storage to [LSO,
> > > ELO]." I
> > > > > > > > mentioned
> > > > > > > > > > an
> > > > > > > > > > > > issue
> > > > > > > > > > > > > > earlier. Suppose the leader's local start offset is
> > > 100.
> > > > > The
> > > > > > > > > > follower
> > > > > > > > > > > > finds
> > > > > > > > > > > > > > a remote segment covering offset range [80, 120).
> > The
> > > > > > > > producerState
> > > > > > > > > > > > with
> > > > > > > > > > > > > > this remote segment is up to offset 120. To trim
> > the
> > > > > > > > producerState
> > > > > > > > > > to
> > > > > > > > > > > > > > offset 100 requires more work since one needs to
> > > > > download the
> > > > > > > > > > previous
> > > > > > > > > > > > > > producerState up to offset 80 and then replay the
> > > > > messages
> > > > > > > > from 80
> > > > > > > > > > to
> > > > > > > > > > > > 100.
> > > > > > > > > > > > > > It seems that it's simpler in this case for the
> > > follower
> > > > > just
> > > > > > > > to
> > > > > > > > > > take
> > > > > > > > > > > > the
> > > > > > > > > > > > > > remote segment as it is and start fetching from
> > > offset
> > > > > 120.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It
> > seems
> > > that
> > > > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called
> > > on the
> > > > > > > > replicas
> > > > > > > > > > for
> > > > > > > > > > > > a
> > > > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > > obtains the metadata for remote segments for
> > > deletion.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > > implemented
> > > > > now.
> > > > > > > > > > Could you
> > > > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > > > following
> > > > > > > two
> > > > > > > > > > APIs
> > > > > > > > > > > > are
> > > > > > > > > > > > > > used. According to the wiki, the former is used for
> > > topic
> > > > > > > > deletion
> > > > > > > > > > and
> > > > > > > > > > > > the
> > > > > > > > > > > > > > latter is used for retention. It seems that
> > retention
> > > > > should
> > > > > > > > use
> > > > > > > > > > the
> > > > > > > > > > > > former
> > > > > > > > > > > > > > since remote segments without a matching epoch in
> > the
> > > > > leader
> > > > > > > > > > > > (potentially
> > > > > > > > > > > > > > due to unclean leader election) also need to be
> > > garbage
> > > > > > > > collected.
> > > > > > > > > > The
> > > > > > > > > > > > > > latter seems to be used for the new leader to
> > > determine
> > > > > the
> > > > > > > > last
> > > > > > > > > > tiered
> > > > > > > > > > > > > > segment.
> > > > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > topicPartition)
> > > > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it
> > > seems
> > > > > that
> > > > > > > > they
> > > > > > > > > > can
> > > > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > > > > instead
> > > > > > > of
> > > > > > > > > > Long?
> > > > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > > > > implementation and
> > > > > > > > > > > > others
> > > > > > > > > > > > > > don't?
> > > > > > > > > > > > > > 5102.4. Could we define
> > > RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> > > pass
> > > > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array
> > > than a
> > > > > file
> > > > > > > > > > since it
> > > > > > > > > > > > will
> > > > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it
> > > needs
> > > > > both
> > > > > > > > > > > > baseOffset and
> > > > > > > > > > > > > > startOffset. For example, deleteRecords() could
> > move
> > > the
> > > > > > > > > > startOffset
> > > > > > > > > > > > to the
> > > > > > > > > > > > > > middle of a segment. If we copy the full segment to
> > > > > remote
> > > > > > > > > > storage, the
> > > > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > is
> > > > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > > > 5103.1 Could we define the default value of
> > > non-required
> > > > > > > > configs
> > > > > > > > > > (e.g
> > > > > > > > > > > > the
> > > > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > > > > default
> > > > > > > to
> > > > > > > > > > > > retention.ms,
> > > > > > > > > > > > > > instead of remote.log.retention.minutes. Similarly,
> > > it
> > > > > seems
> > > > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > > > > segment.bytes.
> > > > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > > > description
> > > > > > > > says
> > > > > > > > > > "used
> > > > > > > > > > > > in
> > > > > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > > > > indexes
> > > > > > > and
> > > > > > > > > > clean
> > > > > > > > > > > > up
> > > > > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > > > > config remote.log.reader.threads for fetching
> > remote
> > > > > data.
> > > > > > > It's
> > > > > > > > > > weird
> > > > > > > > > > > > to
> > > > > > > > > > > > > > fetch remote index and log in different thread
> > pools
> > > > > since
> > > > > > > > both are
> > > > > > > > > > > > used
> > > > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is
> > that
> > > the
> > > > > > > > amount of
> > > > > > > > > > > > time to
> > > > > > > > > > > > > > back off when there is no work to do? If so,
> > perhaps
> > > it
> > > > > can
> > > > > > > be
> > > > > > > > > > renamed
> > > > > > > > > > > > as
> > > > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > > rlm_retry_interval_ms
> > > > > > > > > > configs?
> > > > > > > > > > > > If
> > > > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU)
> > > of the
> > > > > > > index
> > > > > > > > > > files
> > > > > > > > > > > > of
> > > > > > > > > > > > > > remote log segments to avoid multiple index fetches
> > > from
> > > > > the
> > > > > > > > remote
> > > > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on
> > > disk,
> > > > > where
> > > > > > > > is it
> > > > > > > > > > > > stored?
> > > > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> > > Earliest
> > > > > Local
> > > > > > > > > > Offset in
> > > > > > > > > > > > > > different places. It would be useful to standardize
> > > the
> > > > > > > > > > terminology.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux
> > > state". It
> > > > > > > > listed two
> > > > > > > > > > > > options
> > > > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5107. Follower to leader transition: It has step 2,
> > > but
> > > > > not
> > > > > > > > step 1.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5108. If a consumer fetches from the remote data
> > and
> > > the
> > > > > > > remote
> > > > > > > > > > > > storage is
> > > > > > > > > > > > > > not available, what error code is used in the fetch
> > > > > response?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns
> > > the
> > > > > first
> > > > > > > > > > message
> > > > > > > > > > > > > > offset whose timestamp is >= to the given timestamp
> > > in
> > > > > the
> > > > > > > > request.
> > > > > > > > > > > > That
> > > > > > > > > > > > > > means it checks in remote log time indexes first,
> > > after
> > > > > which
> > > > > > > > > > local log
> > > > > > > > > > > > > > time indexes are checked." Could you document which
> > > > > method in
> > > > > > > > RLMM
> > > > > > > > > > is
> > > > > > > > > > > > used
> > > > > > > > > > > > > > for this?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> > > segment
> > > > > > > > metadata of
> > > > > > > > > > that
> > > > > > > > > > > > > > partition with a delete marker and publishes them
> > to
> > > > > RLMM."
> > > > > > > > This
> > > > > > > > > > seems
> > > > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for
> > > the
> > > > > > > > earliest
> > > > > > > > > > leader
> > > > > > > > > > > > > > epoch by calling
> > > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > > topicPartition,
> > > > > > > > > > > > int
> > > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > > offset."
> > > > > Do we
> > > > > > > > need
> > > > > > > > > > that
> > > > > > > > > > > > > > since replication propagates logStartOffset
> > already?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > > > > fetching
> > > > > > > > from
> > > > > > > > > > remote
> > > > > > > > > > > > > > storage?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5113. "Committed offsets can be stored in a local
> > > file to
> > > > > > > avoid
> > > > > > > > > > > > reading the
> > > > > > > > > > > > > > messages again when a broker is restarted." Could
> > you
> > > > > > > describe
> > > > > > > > the
> > > > > > > > > > > > format
> > > > > > > > > > > > > > and the location of the file? Also, could the same
> > > > > message be
> > > > > > > > > > > > processed by
> > > > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> > > handle
> > > > > > > that?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > > > 5114.1 There are two records named
> > > > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > > with
> > > > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > > document
> > > > > > > > whether
> > > > > > > > > > > > endOffset
> > > > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> > > explain
> > > > > > > > > > LeaderEpoch a
> > > > > > > > > > > > bit
> > > > > > > > > > > > > > more? Is that the epoch of the leader when it
> > copies
> > > the
> > > > > > > > segment to
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> > > more?
> > > > > > > Each
> > > > > > > > > > record
> > > > > > > > > > > > in
> > > > > > > > > > > > > > Kafka already has a timestamp field. Could we just
> > > use
> > > > > that?
> > > > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be
> > int32?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> > > with
> > > > > the
> > > > > > > > log
> > > > > > > > > > > > cleaner
> > > > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth
> > like
> > > > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > > > > processes
> > > > > > > > it
> > > > > > > > > > if it
> > > > > > > > > > > > is
> > > > > > > > > > > > > > not yet processed earlier." How does it know
> > whether
> > > > > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5117. Should we add a new MessageFormatter to read
> > > the
> > > > > tier
> > > > > > > > > > metadata
> > > > > > > > > > > > topic?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task
> > > queue
> > > > > size.
> > > > > > > > If
> > > > > > > > > > the
> > > > > > > > > > > > task
> > > > > > > > > > > > > > queue is full, broker will stop reading remote log
> > > > > segments."
> > > > > > > > What
> > > > > > > > > > do
> > > > > > > > > > > > we
> > > > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5119. It would be useful to list all things not
> > > > > supported in
> > > > > > > > the
> > > > > > > > > > first
> > > > > > > > > > > > > > version in a Future work or Limitations section.
> > For
> > > > > example,
> > > > > > > > > > compacted
> > > > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable
> > from
> > > > > true to
> > > > > > > > false,
> > > > > > > > > > > > etc.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > > > > > > > kprakasam@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.duggana@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.duggana@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
> > <
> > > > > > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > > > > > (
> > > > > > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > > > > > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > >
> >

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

Posted by Manikumar <ma...@gmail.com>.
Hi Satish,

Thanks for the KIP.  some of my comments below.

9201. Can we make it clear by mentioning RLM as an internal component and
RLMM and RSM are pluggable components.
It's also good to update/add the RLM, RLMM components  diagrams with
internal components/tasks.

9202. RLMM metadata topic is a non-compact topic. Can we make clear that
metadata topic retention time should be greater
 than user topics. or How about setting retention set to -1 (unlimited) and
let users to modify it.

9203. RLMM has additional responsibility of handling topic delete requests.
With current design, RLMM implementations need
to implement deletion logic. How about making RemotePartitionRemover as a
separate task, which can be reused..

9204. Can we list all the new error codes added by the KIP?

9205. Can we also support  `remote.log.metadata.producr.*`,
`remote.log.metadata.consumer.*` prefixed configs to
override any default configs ?. also include RLMM cache configs, if any.

9206. In the Upgrade section, we have below statement
"Before enabling tiered storage, you should make sure the producer
snapshots are built for all the segments for that topic
in all followers. You should wait till the log retention occurs for all the
segments so that all the segments have producer snapshots."

 Can you add more details about this constraint?


Thanks,
Manikumar

On Tue, Jan 26, 2021 at 1:12 AM Jun Rao <ju...@confluent.io> wrote:

> Hi, Satish,
>
> Thanks for the reply. A few more comments below.
>
> 6000. When RSM/RLMM is not available, it seems that we need to return a
> retriable error to the affected client requests. So, should we add a new
> error code for requests like fetch and listOffset?
>
> 9111. RemotePartitionRemover:
> 91111.1 Could we make it clear that it's part of the default RLMM
> implementation and a separate implementation is needed if one has a
> customized RLMM?
> 91111.2 "RPRM gets all the remote log segments for the partition using RLMM
> and each of these remote log segments is deleted with the next steps.RLMM
> subscribes to the local remote log metadata partitions and it will have the
> segment metadata of all the user topic partitions associated with that
> remote log metadata partition." It seems that RLMM needs to subscribe to
> the remote log metadata partitions first before those segments can be
> deleted?
> 9111.3 There are still references to "remote log cleaners". They need to be
> replaced with RemotePartitionRemover.
>
> 9114.1 Could we add the requirement on log.message.format before enabling
> tiered storage?
>
> 9116. RemoteLogMetadataFormatter: This is used with the ConsoleConsumer
> tool, right? Then, are those new options for ConsoleConsumer and how are
> they passed in?
>
> Thanks,
>
> Jun
>
>
> On Mon, Jan 25, 2021 at 8:02 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi Jun,
> >
> > Thanks for your comments. Please find the inline replies below.
> > Several of these were discussed/clarified in our last discussion.
> >
> > 6000. Since we are returning new error codes, we need to bump up the
> > protocol version for Fetch request. Also, it will be useful to document
> all
> > new error codes and whether they are retriable or not.
> > This previous comment doesn't seem to be addressed.
> >
> > Updated KIP on fetch protocol and error code.
> >
> > 9110. flat_file_format: Could you define the serialized representation
> for
> > each field in the header?
> >
> > Added the serialized representation types in the KIP.
> >
> > 9111. RPM has the following 2 steps.
> > "1. The controller publishes delete_partition_marked event to say that
> the
> > partition is marked for deletion. There can be multiple events published
> > when the controller restarts or failover and this event will be
> > deduplicated by RPM.
> > 2. RPM receives the delete_partition_marked and processes it if it is not
> > yet processed earlier."
> > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> default
> > RLMM implementation or is it meant for any RLMM implementation?
> >
> > RPM(or RPRM) is only for the default RLMM implementation. RPM receives
> > the delete_partition_marked events from RLMM and acts on them. Updated
> > KIP with more details.
> >
> > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to
> make
> > it clear in the comment that this is for the initial retry backoff.
> >
> > Updated the KIP.
> >
> > 9113. RLMM:
> > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> updated.
> >
> > Updated javadoc of this method.
> >
> > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> > since all other update events have leaderEpoch?
> >
> > +1 to have that,  updated the KIP.
> >
> > 9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
> > to make it clear this is for deletion?
> >
> > Sure, updated the KIP.
> >
> > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
> > clear this is for segment?
> >
> > Sure, Updated the KIP.
> >
> > 9114.Upgrade:
> > 9114.1 It seems that we require message format > 0.11 for turning on the
> > remote store feature.
> >
> > 9114.2 It's not clear to me why remote.log.storage.system.enable needs to
> > be set to true before bumping up inter.broker.protocol.version.
> >
> > Agree that based on our discussion, this is not required. Upgrade
> > notes is updated.
> >
> > 9114.3 "If the topic-id is not received in the LeaderAndIsr request then
> > remote log storage will not start. But it will log an error message in
> the
> > log. One way to address this is to do a rolling restart of that broker,
> so
> > that the leader will be moved to another broker and the controller will
> > send LeaderAndIsr with the registered topic-id." Why is this needed? With
> > KIP-516, after upgrading to the latest protocol, topicIds are auto
> created.
> >
> > I thought there may be edge cases of not receiving topic-id but you
> > clarified in the meeting that won’t be the case. I agree that it is
> > not needed.
> >
> > 9115. testing: Could you comment on how we plan to do integration and
> > system tests? Do we plan to include, for example, an in-memory
> > implementation of RSM?
> >
> > For integration tests, we can have file based RSM that we have. For
> > system tests, we can have a single node HDFS cluster in one of the
> > containers and use HDFS RSM implementation.
> >
> > 9116. There is still a todo for the message formatter.
> >
> > Updated the KIP with the format.
> >
> >
> > Satish.
> >
> >
> > On Tue, 12 Jan 2021 at 07:07, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Hi, Satish,
> > >
> > > Thanks for the reply. A few more followup comments.
> > >
> > > 6000. Since we are returning new error codes, we need to bump up the
> > > protocol version for Fetch request. Also, it will be useful to document
> > all
> > > new error codes and whether they are retriable or not.
> > > This previous comment doesn't seem to be addressed.
> > >
> > > 9110. flat_file_format: Could you define the serialized representation
> > for
> > > each field in the header?
> > >
> > > 9111. RPM has the following 2 steps.
> > > "1. The controller publishes delete_partition_marked event to say that
> > the
> > > partition is marked for deletion. There can be multiple events
> published
> > > when the controller restarts or failover and this event will be
> > > deduplicated by RPM.
> > > 2. RPM receives the delete_partition_marked and processes it if it is
> not
> > > yet processed earlier."
> > > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> > default
> > > RLMM implementation or is it meant for any RLMM implementation?
> > >
> > > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to
> > make
> > > it clear in the comment that this is for the initial retry backoff.
> > >
> > > 9113. RLMM:
> > > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> > updated.
> > > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> > > since all other update events have leaderEpoch?
> > > 9113.3 Could we rename RemotePartitionState to
> RemotePartitionDeleteState
> > > to make it clear this is for deletion?
> > > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make
> it
> > > clear this is for segment?
> > >
> > > 9114.Upgrade:
> > > 9114.1 It seems that we require message format > 0.11 for turning on
> the
> > > remote store feature.
> > > 9114.2 It's not clear to me why remote.log.storage.system.enable needs
> to
> > > be set to true before bumping up inter.broker.protocol.version.
> > > 9114.3 "If the topic-id is not received in the LeaderAndIsr request
> then
> > > remote log storage will not start. But it will log an error message in
> > the
> > > log. One way to address this is to do a rolling restart of that broker,
> > so
> > > that the leader will be moved to another broker and the controller will
> > > send LeaderAndIsr with the registered topic-id." Why is this needed?
> With
> > > KIP-516, after upgrading to the latest protocol, topicIds are auto
> > created.
> > >
> > > 9115. testing: Could you comment on how we plan to do integration and
> > > system tests? Do we plan to include, for example, an in-memory
> > > implementation of RSM?
> > >
> > > 9116. There is still a todo for the message formatter.
> > >
> > > Jun
> > >
> > > On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <
> satish.duggana@gmail.com>
> > > wrote:
> > >
> > > > Hi Jun,
> > > > Thanks for your comments. Please find the inline replies below.
> > > >
> > > > 6022. For packages used for server plugins, the convention is to
> > > > use org.apache.kafka.server. See java-based Authorizer as an example.
> > > >
> > > > Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> > > > ‘org.apache.kafka.server.log.remote.storage’.  Updated in the KIP.
> > > >
> > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The
> > flat
> > > > file captures the state of the remote segments. After a partition is
> > > > deleted, it seems that we just need to remove the partitions's remote
> > > > segments from the flat file.
> > > >
> > > > DeletePartitionState might not yet have been processed by RPM and not
> > > > completed. We  will not have  that in flat  file format  once it
> > > > reaches DELETE_PARTITION_FINISHED state.
> > > >
> > > > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > > > version. It seems that's doable. One can just do the normal upgrade
> > first
> > > > and wait enough time (for producer snapshots to be built), and then
> > enable
> > > > remote storage.
> > > >
> > > > Upgrade notes updates in the KIP.
> > > >
> > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > > > tracking the remote segments when RLMM.onPartitionLeadershipChanges()
> > is
> > > > called with the broker being the leader for __remote_log_metadata
> > > > partition? If so, could we document it?
> > > >
> > > > The current plan is to have that as part of RLMM and RPM uses that to
> > > > get the remote segments list. I will add this detail in the
> respective
> > > > sections.
> > > >
> > > > Satish.
> > > >
> > > > On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the reply. A few more followup comments.
> > > > >
> > > > > 6022. For packages used for server plugins, the convention is to
> > > > > use org.apache.kafka.server. See java-based Authorizer as an
> example.
> > > > >
> > > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format?
> The
> > flat
> > > > > file captures the state of the remote segments. After a partition
> is
> > > > > deleted, it seems that we just need to remove the partitions's
> remote
> > > > > segments from the flat file.
> > > > >
> > > > > 9101. Upgrade: It will be useful to allow direct upgrade from an
> old
> > > > > version. It seems that's doable. One can just do the normal upgrade
> > first
> > > > > and wait enough time (for producer snapshots to be built), and then
> > > > enable
> > > > > remote storage.
> > > > >
> > > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM
> starts
> > > > > tracking the remote segments when
> > RLMM.onPartitionLeadershipChanges() is
> > > > > called with the broker being the leader for __remote_log_metadata
> > > > > partition? If so, could we document it?
> > > > >
> > > > > Jun
> > > > >
> > > > > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <
> > kprakasam@confluent.io
> > > > >
> > > > > wrote:
> > > > >
> > > > > > Hi Satish,
> > > > > >
> > > > > > Thanks for the updates! A few more comments below.
> > > > > >
> > > > > > 9001. Under the "Upgrade" section, there is a line mentioning:
> > > > "Upgrade the
> > > > > > existing Kafka cluster to 2.7 version and allow this to run for
> > the log
> > > > > > retention of user topics that you want to enable tiered storage.
> > This
> > > > will
> > > > > > allow all the topics to have the producer snapshots generated for
> > each
> > > > log
> > > > > > segment." -- Which associated change in AK were you referring to
> > here?
> > > > Is
> > > > > > it: https://github.com/apache/kafka/pull/7929 ? It seems like I
> > don't
> > > > see
> > > > > > it in the 2.7 release branch yet, here is the link:
> > > > > > https://github.com/apache/kafka/commits/2.7.
> > > > > >
> > > > > > 9002. Under the "Upgrade" section, the configuration mentioned is
> > > > > > 'remote.log.storage.system.enable'. However, under "Public
> > Interfaces"
> > > > > > section the corresponding configuration is
> > > > 'remote.storage.system.enable'.
> > > > > > Could we use the same one in both, maybe
> > > > > > 'remote.log.storage.system.enable'?
> > > > > >
> > > > > > 9003. Under "Per Topic Configuration", the KIP recommends setting
> > > > > > 'remote.log.storage.enable' to true at a per-topic level. It will
> > be
> > > > useful
> > > > > > to add a line that if the user wants to enable it for all topics,
> > then
> > > > they
> > > > > > should be able to set the cluster-wide default to true. Also, it
> > will
> > > > be
> > > > > > useful to mention that the KIP currently does not support setting
> > it to
> > > > > > false (after it is set to true), and add that to the future work
> > > > section.
> > > > > >
> > > > > > 9004. Under "Committed offsets file format", the sample provided
> > shows
> > > > > > partition number and offset. Is the topic name required for
> > identifying
> > > > > > which topic the partitions belong to?
> > > > > >
> > > > > > 9005. Under "Internal flat-file store format of remote log
> > metadata",
> > > > it
> > > > > > seems useful to specify both topic name and topic ID for
> debugging
> > > > > > purposes.
> > > > > >
> > > > > > 9006. Under "Internal flat-file store format of remote log
> > metadata",
> > > > the
> > > > > > description of "metadata-topic-offset" currently says "offset of
> > the
> > > > remote
> > > > > > log metadata topic from which this topic partition's remote log
> > > > metadata is
> > > > > > fetched." Just for the wording, perhaps you meant to refer to the
> > > > offset
> > > > > > upto which the file has been committed? i.e. "offset of the
> remote
> > log
> > > > > > metadata topic upto which this topic partition's remote log
> > metadata
> > > > has
> > > > > > been committed into this file."
> > > > > >
> > > > > > 9007. Under "Internal flat-file store format of remote log
> > metadata",
> > > > the
> > > > > > schema of the payload (i.e. beyond the header) seems to contain
> the
> > > > events
> > > > > > from the metadata topic. It seems useful to instead persist the
> > > > > > representation of the materialized state of the events, so that
> > for the
> > > > > > same segment only the latest state is stored. Besides reducing
> > storage
> > > > > > footprint, this also is likely to relate directly with the
> > in-memory
> > > > > > representation of the RLMM cache (which probably is some kind of
> a
> > Map
> > > > with
> > > > > > key being segment ID and value being the segment state), so
> > recovery
> > > > from
> > > > > > disk will be straightforward.
> > > > > >
> > > > > > 9008. Under "Topic deletion lifecycle", step (1), it will be
> > useful to
> > > > > > mention when in the deletion flow does the controller publish the
> > > > > > delete_partition_marked event to say that the partition is marked
> > for
> > > > > > deletion?
> > > > > >
> > > > > > 9009. There are ~4 TODOs in the KIP. Could you please address
> > these or
> > > > > > remove them?
> > > > > >
> > > > > > 9010. There is a reference to a Google doc on the KIP which was
> > used
> > > > > > earlier for discussions. Please could you remove the reference,
> > since
> > > > the
> > > > > > KIP is the source of the truth?
> > > > > >
> > > > > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > > > > >
> > > > > >
> > > > > > Cheers,
> > > > > > Kowshik
> > > > > >
> > > > > >
> > > > > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Jun,
> > > > > > > Thanks for your comments. Please go through the inline replies.
> > > > > > >
> > > > > > >
> > > > > > > 5102.2: It seems that both positions can just be int. Another
> > option
> > > > is
> > > > > > to
> > > > > > > have two methods. Would it be clearer?
> > > > > > >
> > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata,  int startPosition)
> > > > > > throwsRemoteStorageException;
> > > > > > >
> > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata, int startPosition, int endPosition)
> > throws
> > > > > > > RemoteStorageException;
> > > > > > >
> > > > > > > That makes sense to me, updated the KIP.
> > > > > > >
> > > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > log.retention.bytes
> > > > > > > to be consistent with local.log.retention.ms?
> > > > > > >
> > > > > > > Yes, it can be defaulted to log.retention.bytes.
> > > > > > >
> > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > >
> > > > > > > Added TopicIdPartition in the KIP.
> > > > > > >
> > > > > > > 6022: For all public facing classes, could you specify the
> > package
> > > > name?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the reply. A few more comments below.
> > > > > > > >
> > > > > > > > 5102.2: It seems that both positions can just be int. Another
> > > > option is
> > > > > > > to
> > > > > > > > have two methods. Would it be clearer?
> > > > > > > >
> > > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata,
> > > > > > > >                                     int startPosition) throws
> > > > > > > > RemoteStorageException;
> > > > > > > >
> > > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata,
> > > > > > > >                                     int startPosition, int
> > > > endPosition)
> > > > > > > > throws RemoteStorageException;
> > > > > > > >
> > > > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > > > >
> > > > > > > > 6010: What kind of tiering throughput have you seen with 5
> > threads?
> > > > > > > >
> > > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > > log.retention.bytes
> > > > > > > > to be consistent with local.log.retention.ms?
> > > > > > > >
> > > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > > >
> > > > > > > > 6022: For all public facing classes, could you specify the
> > package
> > > > > > name?
> > > > > > > >
> > > > > > > > It seems that you already added the topicId support. Two
> other
> > > > > > remaining
> > > > > > > > items are (a) the format of local tier metadata storage and
> (b)
> > > > > > upgrade.
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > > > > satish.duggana@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Jun,
> > > > > > > > > Thanks for your comments. Please find the inline replies
> > below.
> > > > > > > > >
> > > > > > > > > >605.2 It's rare for the follower to need the remote data.
> > So,
> > > > the
> > > > > > > current
> > > > > > > > > approach is fine too. Could you document the process of
> > > > rebuilding
> > > > > > the
> > > > > > > > > producer state since we can't simply trim the producerState
> > to an
> > > > > > > offset in
> > > > > > > > > the middle of a segment.
> > > > > > > > >
> > > > > > > > > Will clarify in the KIP.
> > > > > > > > >
> > > > > > > > > >5102.2 Would it be clearer to make startPosiont long and
> > > > endPosition
> > > > > > > of
> > > > > > > > > Optional<Long>?
> > > > > > > > >
> > > > > > > > > We will have arg checks with respective validation. It is
> > not a
> > > > good
> > > > > > > > > practice to have arguments with optional as mentioned here.
> > > > > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as File
> > > > instead of
> > > > > > > > > ByteBuffer.
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >5102.7 Could you define all public methods for
> > LogSegmentData?
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >5103.5 Could you change the reference to
> > > > rlm_process_interval_ms and
> > > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> > retry
> > > > > > interval
> > > > > > > > > config seems still missing. It would be useful to support
> > > > exponential
> > > > > > > > > backoff with the retry interval config.
> > > > > > > > >
> > > > > > > > > Good point. We wanted the retry with truncated exponential
> > > > backoff,
> > > > > > > > > updated the KIP.
> > > > > > > > >
> > > > > > > > > >5111. "RLM follower fetches the earliest offset for the
> > earliest
> > > > > > > leader
> > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > topicPartition,
> > > > > > > int
> > > > > > > > > leaderEpoch) and updates that as the log start offset."
> This
> > > > text is
> > > > > > > still
> > > > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >5115. There are still references to "remote log cleaners".
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >6000. Since we are returning new error codes, we need to
> > bump
> > > > up the
> > > > > > > > > protocol version for Fetch request. Also, it will be useful
> > to
> > > > > > > document all
> > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > >
> > > > > > > > > Sure, we will add that in the KIP.
> > > > > > > > >
> > > > > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs():
> > Currently,
> > > > > > > leaderEpoch
> > > > > > > > > is int32 instead of long.
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> needed
> > > > given
> > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > >
> > > > > > > > > No, it is fixed.
> > > > > > > > >
> > > > > > > > > >6003. RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > topicPartition, long offset, int epochForOffset): Should
> this
> > > > return
> > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > >
> > > > > > > > > That makes sense, updated.
> > > > > > > > >
> > > > > > > > > >6005. RemoteLogState: It seems it's better to split it
> > between
> > > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> > since
> > > > the
> > > > > > > states
> > > > > > > > > are never shared between the two use cases.
> > > > > > > > >
> > > > > > > > > Agree with that, updated.
> > > > > > > > >
> > > > > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > > > However,
> > > > > > > is it
> > > > > > > > > ture that other than the metadata topic, RLMM just needs to
> > know
> > > > > > > whether
> > > > > > > > > there is a replica assigned to this broker and doesn't need
> > to
> > > > know
> > > > > > > whether
> > > > > > > > > the replica is the leader or the follower?
> > > > > > > > >
> > > > > > > > > That may be true. If the implementation does not need that,
> > it
> > > > can
> > > > > > > > > ignore the information in the callback.
> > > > > > > > >
> > > > > > > > > >6007: "Handle expired remote segments (leader and
> > follower)":
> > > > Why is
> > > > > > > this
> > > > > > > > > needed in both the leader and the follower?
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > > > > >                 "type": "int64",
> > > > > > > > > The segment size can just be int32.
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >6009. For the record format in the log, it seems that we
> > need
> > > > to add
> > > > > > > > > record
> > > > > > > > > type and record version before the serialized bytes. We can
> > > > follow
> > > > > > the
> > > > > > > > > convention used in
> > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > >
> > > > > > > > > Yes, KIP already mentions that these are serialized before
> > the
> > > > > > payload
> > > > > > > > > as below. We will mention explicitly that these two are
> > written
> > > > > > before
> > > > > > > > > the data is written.
> > > > > > > > >
> > > > > > > > > RLMM instance on broker publishes the message to the topic
> > with
> > > > key
> > > > > > as
> > > > > > > > > null and value with the below format.
> > > > > > > > >
> > > > > > > > > type      : unsigned var int, represents the value type.
> This
> > > > value
> > > > > > is
> > > > > > > > > 'apikey' as mentioned in the schema.
> > > > > > > > > version : unsigned var int, the 'version' number of the
> type
> > as
> > > > > > > > > mentioned in the schema.
> > > > > > > > > data      : record payload in kafka protocol message
> format.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > >6010. remote.log.manager.thread.pool.size: The default
> > value is
> > > > 10.
> > > > > > > This
> > > > > > > > > might be too high when enabling the tiered feature for the
> > first
> > > > > > time.
> > > > > > > > > Since there are lots of segments that need to be tiered
> > > > initially, a
> > > > > > > large
> > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > >
> > > > > > > > > Is the default value 5 reasonable?
> > > > > > > > >
> > > > > > > > > 6011. "The number of milli seconds to keep the local log
> > segment
> > > > > > > before it
> > > > > > > > > gets deleted. If not set, the value in
> > `log.retention.minutes` is
> > > > > > > used. If
> > > > > > > > > set to -1, no time limit is applied." We should use
> > > > log.retention.ms
> > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > Nice typo catch. Updated the KIP.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Satish.
> > > > > > > > >
> > > > > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > > > >
> > > > > > > > > > Hi, Satish,
> > > > > > > > > >
> > > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > > >
> > > > > > > > > > 605.2 It's rare for the follower to need the remote data.
> > So,
> > > > the
> > > > > > > current
> > > > > > > > > > approach is fine too. Could you document the process of
> > > > rebuilding
> > > > > > > the
> > > > > > > > > > producer state since we can't simply trim the
> > producerState to
> > > > an
> > > > > > > offset
> > > > > > > > > in
> > > > > > > > > > the middle of a segment.
> > > > > > > > > >
> > > > > > > > > > 5102.2 Would it be clearer to make startPosiont long and
> > > > > > endPosition
> > > > > > > of
> > > > > > > > > > Optional<Long>?
> > > > > > > > > >
> > > > > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File
> > > > instead of
> > > > > > > > > > ByteBuffer.
> > > > > > > > > >
> > > > > > > > > > 5102.7 Could you define all public methods for
> > LogSegmentData?
> > > > > > > > > >
> > > > > > > > > > 5103.5 Could you change the reference to
> > > > rlm_process_interval_ms
> > > > > > and
> > > > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> > retry
> > > > > > > interval
> > > > > > > > > > config seems still missing. It would be useful to support
> > > > > > exponential
> > > > > > > > > > backoff with the retry interval config.
> > > > > > > > > >
> > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > earliest
> > > > > > > leader
> > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > > int
> > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> > This
> > > > text
> > > > > > is
> > > > > > > > > still
> > > > > > > > > > there. Also, could we remove earliestLogOffset() from
> RLMM?
> > > > > > > > > >
> > > > > > > > > > 5115. There are still references to "remote log
> cleaners".
> > > > > > > > > >
> > > > > > > > > > 6000. Since we are returning new error codes, we need to
> > bump
> > > > up
> > > > > > the
> > > > > > > > > > protocol version for Fetch request. Also, it will be
> > useful to
> > > > > > > document
> > > > > > > > > all
> > > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > > >
> > > > > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs():
> > Currently,
> > > > > > > > > leaderEpoch
> > > > > > > > > > is int32 instead of long.
> > > > > > > > > >
> > > > > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> > needed
> > > > given
> > > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > > >
> > > > > > > > > > 6003. RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > topicPartition, long offset, int epochForOffset): Should
> > this
> > > > > > return
> > > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > > >
> > > > > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful
> to
> > > > pick a
> > > > > > > more
> > > > > > > > > > indicative name so that people understand what epoch this
> > is.
> > > > > > > > > >
> > > > > > > > > > 6005. RemoteLogState: It seems it's better to split it
> > between
> > > > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> > since
> > > > the
> > > > > > > states
> > > > > > > > > > are never shared between the two use cases.
> > > > > > > > > >
> > > > > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be
> ok.
> > > > However,
> > > > > > > is it
> > > > > > > > > > ture that other than the metadata topic, RLMM just needs
> to
> > > > know
> > > > > > > whether
> > > > > > > > > > there is a replica assigned to this broker and doesn't
> > need to
> > > > know
> > > > > > > > > whether
> > > > > > > > > > the replica is the leader or the follower?
> > > > > > > > > >
> > > > > > > > > > 6007: "Handle expired remote segments (leader and
> > follower)":
> > > > Why
> > > > > > is
> > > > > > > this
> > > > > > > > > > needed in both the leader and the follower?
> > > > > > > > > >
> > > > > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > > > > >                 "type": "int64",
> > > > > > > > > > The segment size can just be int32.
> > > > > > > > > >
> > > > > > > > > > 6009. For the record format in the log, it seems that we
> > need
> > > > to
> > > > > > add
> > > > > > > > > record
> > > > > > > > > > type and record version before the serialized bytes. We
> can
> > > > follow
> > > > > > > the
> > > > > > > > > > convention used in
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > > .
> > > > > > > > > >
> > > > > > > > > > 6010. remote.log.manager.thread.pool.size: The default
> > value
> > > > is 10.
> > > > > > > This
> > > > > > > > > > might be too high when enabling the tiered feature for
> the
> > > > first
> > > > > > > time.
> > > > > > > > > > Since there are lots of segments that need to be tiered
> > > > initially,
> > > > > > a
> > > > > > > > > large
> > > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > > >
> > > > > > > > > > 6011. "The number of milli seconds to keep the local log
> > > > segment
> > > > > > > before
> > > > > > > > > it
> > > > > > > > > > gets deleted. If not set, the value in
> > `log.retention.minutes`
> > > > is
> > > > > > > used.
> > > > > > > > > If
> > > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > > log.retention.ms
> > > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi,
> > > > > > > > > > > We updated the KIP with the points mentioned in the
> > earlier
> > > > mail
> > > > > > > > > > > except for KIP-516 related changes. You can go through
> > them
> > > > and
> > > > > > > let us
> > > > > > > > > > > know if you have any comments. We will update the KIP
> > with
> > > > the
> > > > > > > > > > > remaining todo items and KIP-516 related changes by end
> > of
> > > > this
> > > > > > > > > > > week(5th Dec).
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Satish.
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > Thanks for your comments. Please find the inline
> > replies
> > > > below.
> > > > > > > > > > > >
> > > > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting
> > the
> > > > leader
> > > > > > > epoch
> > > > > > > > > > > > sequence received from remote storage to [LSO,
> ELO]." I
> > > > > > > mentioned an
> > > > > > > > > > > issue
> > > > > > > > > > > > earlier. Suppose the leader's local start offset is
> > 100.
> > > > The
> > > > > > > follower
> > > > > > > > > > > finds
> > > > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > > > producerState
> > > > > > > > > with
> > > > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > > > producerState to
> > > > > > > > > > > > offset 100 requires more work since one needs to
> > download
> > > > the
> > > > > > > > > previous
> > > > > > > > > > > > producerState up to offset 80 and then replay the
> > messages
> > > > from
> > > > > > > 80 to
> > > > > > > > > > > 100.
> > > > > > > > > > > > It seems that it's simpler in this case for the
> > follower
> > > > just
> > > > > > to
> > > > > > > > > take the
> > > > > > > > > > > > remote segment as it is and start fetching from
> offset
> > 120.
> > > > > > > > > > > >
> > > > > > > > > > > > We chose that approach to avoid any edge cases here.
> It
> > > > may be
> > > > > > > > > > > > possible that the remote log segment that is received
> > may
> > > > not
> > > > > > > have
> > > > > > > > > the
> > > > > > > > > > > > same leader epoch sequence from 100-120 as it
> contains
> > on
> > > > the
> > > > > > > > > > > > leader(this can happen due to unclean leader). It is
> > safe
> > > > to
> > > > > > > start
> > > > > > > > > > > > from what the leader returns here.Another way is to
> > find
> > > > the
> > > > > > > remote
> > > > > > > > > > > > log segment
> > > > > > > > > > > >
> > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems
> > that
> > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on
> > the
> > > > > > > replicas
> > > > > > > > > for a
> > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > obtains the metadata for remote segments for
> deletion.
> > > > > > > > > > > >
> > > > > > > > > > > > RLMM will always receive the callback for the remote
> > log
> > > > > > metadata
> > > > > > > > > > > > topic partitions hosted on the local broker and these
> > will
> > > > be
> > > > > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > implemented
> > > > now.
> > > > > > > Could
> > > > > > > > > you
> > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > >
> > > > > > > > > > > > We mentioned KIP-516 and how it helps. We will update
> > this
> > > > KIP
> > > > > > > with
> > > > > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > > > > >
> > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > > following two
> > > > > > > APIs
> > > > > > > > > are
> > > > > > > > > > > > used. According to the wiki, the former is used for
> > topic
> > > > > > > deletion
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > latter is used for retention. It seems that retention
> > > > should
> > > > > > use
> > > > > > > the
> > > > > > > > > > > former
> > > > > > > > > > > > since remote segments without a matching epoch in the
> > > > leader
> > > > > > > > > (potentially
> > > > > > > > > > > > due to unclean leader election) also need to be
> garbage
> > > > > > > collected.
> > > > > > > > > The
> > > > > > > > > > > > latter seems to be used for the new leader to
> > determine the
> > > > > > last
> > > > > > > > > tiered
> > > > > > > > > > > > segment.
> > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > >
> > > > > > > > > > > > Right,.that is what we are currently doing. We will
> > update
> > > > the
> > > > > > > > > > > > javadocs and wiki with that. Earlier, we did not want
> > to
> > > > remove
> > > > > > > the
> > > > > > > > > > > > segments which are not matched with leader epochs
> from
> > the
> > > > > > ladder
> > > > > > > > > > > > partition as they may be used later by a replica
> which
> > can
> > > > > > > become a
> > > > > > > > > > > > leader (unclean leader election) and refer those
> > segments.
> > > > But
> > > > > > > that
> > > > > > > > > > > > may leak these segments in remote storage until the
> > topic
> > > > > > > lifetime.
> > > > > > > > > We
> > > > > > > > > > > > decided to cleanup the segments with the oldest
> incase
> > of
> > > > size
> > > > > > > based
> > > > > > > > > > > > retention also.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it
> seems
> > > > that
> > > > > > > they can
> > > > > > > > > > > > use RemoteLogSegmentId instead of
> > RemoteLogSegmentMetadata.
> > > > > > > > > > > >
> > > > > > > > > > > > It will be useful to have metadata for RSM to fetch
> log
> > > > > > segment.
> > > > > > > It
> > > > > > > > > > > > may create location/path using id with other metadata
> > too.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > > > instead of
> > > > > > > Long?
> > > > > > > > > > > >
> > > > > > > > > > > > Wanted to keep endPosition as optional to read till
> the
> > > > end of
> > > > > > > the
> > > > > > > > > > > > segment and avoid sentinels.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > implementation
> > > > > > > and
> > > > > > > > > > > others
> > > > > > > > > > > > Don't?
> > > > > > > > > > > >
> > > > > > > > > > > > Actually,  RSM will not have any default
> > implementations.
> > > > > > Those 3
> > > > > > > > > > > > methods were made default earlier for tests etc.
> > Updated
> > > > the
> > > > > > > wiki.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.4. Could we define
> RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, they will be added.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> > pass
> > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array
> than
> > a
> > > > file
> > > > > > > since
> > > > > > > > > it
> > > > > > > > > > > will
> > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > >
> > > > > > > > > > > > Right, this is in plan.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it
> needs
> > > > both
> > > > > > > > > baseOffset
> > > > > > > > > > > and
> > > > > > > > > > > > startOffset. For example, deleteRecords() could move
> > the
> > > > > > > startOffset
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > middle of a segment. If we copy the full segment to
> > remote
> > > > > > > storage,
> > > > > > > > > the
> > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > >
> > > > > > > > > > > > Good point. startOffset is baseOffset by default, if
> > not
> > > > set
> > > > > > > > > explicitly.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, updated the wiki.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > >
> > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > >
> > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > 5103.1 Could we define the default value of
> > non-required
> > > > > > configs
> > > > > > > > > (e.g the
> > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, that makes sense.
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > > > default to
> > > > > > > > > > > retention.ms,
> > > > > > > > > > > > instead of remote.log.retention.minutes. Similarly,
> it
> > > > seems
> > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > segment.bytes.
> > > > > > > > > > > >
> > > > > > > > > > > > Right, we do not have  remote.log.retention as we
> > discussed
> > > > > > > earlier.
> > > > > > > > > > > > Thanks for catching the typo.
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > description
> > > > > > says
> > > > > > > > > "used in
> > > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > > > indexes and
> > > > > > > > > clean up
> > > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > > config remote.log.reader.threads for fetching remote
> > data.
> > > > It's
> > > > > > > > > weird to
> > > > > > > > > > > > fetch remote index and log in different thread pools
> > since
> > > > both
> > > > > > > are
> > > > > > > > > used
> > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > >
> > > > > > > > > > > > Right, remote.log.manager.thread.pool is mainly used
> > for
> > > > > > > copy/cleanup
> > > > > > > > > > > > activities. Fetch path always goes through
> > > > > > > remote.log.reader.threads.
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that
> > the
> > > > amount
> > > > > > > of
> > > > > > > > > time
> > > > > > > > > > > to
> > > > > > > > > > > > back off when there is no work to do? If so, perhaps
> it
> > > > can be
> > > > > > > > > renamed as
> > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > >
> > > > > > > > > > > > This is the delay interval for each iteration. It may
> > be
> > > > > > renamed
> > > > > > > to
> > > > > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > rlm_retry_interval_ms
> > > > > > > > > configs? If
> > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > >
> > > > > > > > > > > > remote.log.manager.task.interval.ms is the process
> > > > internal,
> > > > > > > retry
> > > > > > > > > > > > interval is missing in the configs, which will be
> > updated
> > > > in
> > > > > > the
> > > > > > > KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of
> > the
> > > > index
> > > > > > > > > files of
> > > > > > > > > > > > remote log segments to avoid multiple index fetches
> > from
> > > > the
> > > > > > > remote
> > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on
> disk,
> > > > where is
> > > > > > > it
> > > > > > > > > > > stored?
> > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > >
> > > > > > > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > > > > > > `remote-log-index-cache` under log dir. We plan to
> > have a
> > > > > > config
> > > > > > > for
> > > > > > > > > > > > that instead of default. We will have a configuration
> > for
> > > > that.
> > > > > > > > > > > >
> > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> Earliest
> > > > Local
> > > > > > > Offset
> > > > > > > > > in
> > > > > > > > > > > > different places. It would be useful to standardize
> the
> > > > > > > terminology.
> > > > > > > > > > > >
> > > > > > > > > > > > Sure.
> > > > > > > > > > > >
> > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state".
> > It
> > > > listed
> > > > > > > two
> > > > > > > > > > > options
> > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > We already mentioned in the KIP that we chose
> option-2.
> > > > > > > > > > > >
> > > > > > > > > > > > 5107. Follower to leader transition: It has step 2,
> > but not
> > > > > > step
> > > > > > > 1.
> > > > > > > > > > > > Step-1 is there but it is not explicitly highlighted.
> > It is
> > > > > > > previous
> > > > > > > > > > > > table to step-2.
> > > > > > > > > > > >
> > > > > > > > > > > > 5108. If a consumer fetches from the remote data and
> > the
> > > > remote
> > > > > > > > > storage
> > > > > > > > > > > is
> > > > > > > > > > > > not available, what error code is used in the fetch
> > > > response?
> > > > > > > > > > > >
> > > > > > > > > > > > Good point. We have not yet defined the error for
> this
> > > > case. We
> > > > > > > need
> > > > > > > > > > > > to define an error message and send the same in fetch
> > > > response.
> > > > > > > > > > > >
> > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns
> the
> > > > first
> > > > > > > message
> > > > > > > > > > > > offset whose timestamp is >= to the given timestamp
> in
> > the
> > > > > > > request.
> > > > > > > > > That
> > > > > > > > > > > > means it checks in remote log time indexes first,
> after
> > > > which
> > > > > > > local
> > > > > > > > > log
> > > > > > > > > > > > time indexes are checked." Could you document which
> > method
> > > > in
> > > > > > > RLMM is
> > > > > > > > > > > used
> > > > > > > > > > > > for this?
> > > > > > > > > > > >
> > > > > > > > > > > > Okay.
> > > > > > > > > > > >
> > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> segment
> > > > metadata
> > > > > > > of
> > > > > > > > > that
> > > > > > > > > > > > partition with a delete marker and publishes them to
> > RLMM."
> > > > > > This
> > > > > > > > > seems
> > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > >
> > > > > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > > > > >
> > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for
> the
> > > > > > earliest
> > > > > > > > > leader
> > > > > > > > > > > > epoch by calling
> RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > topicPartition,
> > > > > > > > > > > int
> > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > offset." Do
> > > > we
> > > > > > > need
> > > > > > > > > that
> > > > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > > > >
> > > > > > > > > > > > Good point. Right, existing replication protocol
> takes
> > > > care of
> > > > > > > > > > > > updating the followers’s log start offset received
> > from the
> > > > > > > leader.
> > > > > > > > > > > >
> > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > fetching
> > > > > > from
> > > > > > > > > remote
> > > > > > > > > > > > storage?
> > > > > > > > > > > >
> > > > > > > > > > > > Remote reads may fail within the current default wait
> > > > time, but
> > > > > > > > > > > > subsequent fetches would be able to serve as that
> data
> > is
> > > > > > stored
> > > > > > > in
> > > > > > > > > > > > the local cache. This cache is currently implemented
> in
> > > > RSMs.
> > > > > > > But we
> > > > > > > > > > > > plan to pull this into the remote log messaging layer
> > in
> > > > > > future.
> > > > > > > > > > > >
> > > > > > > > > > > > 5113. "Committed offsets can be stored in a local
> file
> > to
> > > > avoid
> > > > > > > > > reading
> > > > > > > > > > > the
> > > > > > > > > > > > messages again when a broker is restarted." Could you
> > > > describe
> > > > > > > the
> > > > > > > > > format
> > > > > > > > > > > > and the location of the file? Also, could the same
> > message
> > > > be
> > > > > > > > > processed
> > > > > > > > > > > by
> > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> > handle
> > > > that?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > 5114.1 There are two records named
> > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > with
> > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > >
> > > > > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > > > > >
> > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > document
> > > > > > whether
> > > > > > > > > > > endOffset
> > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > >
> > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> > explain
> > > > > > > LeaderEpoch
> > > > > > > > > a
> > > > > > > > > > > bit
> > > > > > > > > > > > more? Is that the epoch of the leader when it copies
> > the
> > > > > > segment
> > > > > > > to
> > > > > > > > > > > remote
> > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > >
> > > > > > > > > > > > Right, this is the leader epoch of the broker which
> > copied
> > > > this
> > > > > > > > > > > > segment. This is helpful in reason about which broker
> > > > copied
> > > > > > the
> > > > > > > > > > > > segment to remote storage.
> > > > > > > > > > > >
> > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> > more?
> > > > Each
> > > > > > > > > record in
> > > > > > > > > > > > Kafka already has a timestamp field. Could we just
> use
> > > > that?
> > > > > > > > > > > >
> > > > > > > > > > > > This is the  timestamp at which  the respective event
> > > > occurred.
> > > > > > > Added
> > > > > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any
> > other
> > > > > > > > > > > > implementation. We thought about that but it looked
> > > > cleaner to
> > > > > > > use at
> > > > > > > > > > > > the message structure level instead of getting that
> > from
> > > > the
> > > > > > > consumer
> > > > > > > > > > > > record and using that to build the respective event.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > > > >
> > > > > > > > > > > > Right, it looks like config allows only int value >=
> > 14.
> > > > > > > > > > > >
> > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> > with
> > > > the
> > > > > > log
> > > > > > > > > cleaner
> > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > >
> > > > > > > > > > > > I am fine with RemotePartitionRemover or
> > > > > > > RemoteLogDeletionManager(we
> > > > > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > > > > >
> > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > > > processes
> > > > > > it
> > > > > > > if
> > > > > > > > > it is
> > > > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > > >
> > > > > > > > > > > > This is to handle duplicate delete_partition_marked
> > > > events. RLC
> > > > > > > > > > > > internally maintains a state for the delete_partition
> > > > events
> > > > > > and
> > > > > > > if
> > > > > > > > > it
> > > > > > > > > > > > already has an existing event then it ignores if it
> is
> > > > already
> > > > > > > being
> > > > > > > > > > > > processed.
> > > > > > > > > > > >
> > > > > > > > > > > > 5117. Should we add a new MessageFormatter to read
> the
> > tier
> > > > > > > metadata
> > > > > > > > > > > topic?
> > > > > > > > > > > >
> > > > > > > > > > > > Right, this is in plan but did not mention it in the
> > KIP.
> > > > This
> > > > > > > will
> > > > > > > > > be
> > > > > > > > > > > > useful for debugging purposes too.
> > > > > > > > > > > >
> > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task
> queue
> > > > size.
> > > > > > If
> > > > > > > the
> > > > > > > > > task
> > > > > > > > > > > > queue is full, broker will stop reading remote log
> > > > segments."
> > > > > > > What
> > > > > > > > > do we
> > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > >
> > > > > > > > > > > > We return an error response for that partition.
> > > > > > > > > > > >
> > > > > > > > > > > > 5119. It would be useful to list all things not
> > supported
> > > > in
> > > > > > the
> > > > > > > > > first
> > > > > > > > > > > > version in a Future work or Limitations section. For
> > > > example,
> > > > > > > > > compacted
> > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> > true
> > > > to
> > > > > > > false,
> > > > > > > > > etc.
> > > > > > > > > > > >
> > > > > > > > > > > > We already have a non-goals section which is filled
> > with
> > > > some
> > > > > > of
> > > > > > > > > these
> > > > > > > > > > > > details. Do we need another limitations section?
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Satish.
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <
> > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > below.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 605.2 "Build the local leader epoch cache by
> cutting
> > the
> > > > > > leader
> > > > > > > > > epoch
> > > > > > > > > > > > > sequence received from remote storage to [LSO,
> > ELO]." I
> > > > > > > mentioned
> > > > > > > > > an
> > > > > > > > > > > issue
> > > > > > > > > > > > > earlier. Suppose the leader's local start offset is
> > 100.
> > > > The
> > > > > > > > > follower
> > > > > > > > > > > finds
> > > > > > > > > > > > > a remote segment covering offset range [80, 120).
> The
> > > > > > > producerState
> > > > > > > > > > > with
> > > > > > > > > > > > > this remote segment is up to offset 120. To trim
> the
> > > > > > > producerState
> > > > > > > > > to
> > > > > > > > > > > > > offset 100 requires more work since one needs to
> > > > download the
> > > > > > > > > previous
> > > > > > > > > > > > > producerState up to offset 80 and then replay the
> > > > messages
> > > > > > > from 80
> > > > > > > > > to
> > > > > > > > > > > 100.
> > > > > > > > > > > > > It seems that it's simpler in this case for the
> > follower
> > > > just
> > > > > > > to
> > > > > > > > > take
> > > > > > > > > > > the
> > > > > > > > > > > > > remote segment as it is and start fetching from
> > offset
> > > > 120.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It
> seems
> > that
> > > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called
> > on the
> > > > > > > replicas
> > > > > > > > > for
> > > > > > > > > > > a
> > > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > obtains the metadata for remote segments for
> > deletion.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > implemented
> > > > now.
> > > > > > > > > Could you
> > > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > > following
> > > > > > two
> > > > > > > > > APIs
> > > > > > > > > > > are
> > > > > > > > > > > > > used. According to the wiki, the former is used for
> > topic
> > > > > > > deletion
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > > latter is used for retention. It seems that
> retention
> > > > should
> > > > > > > use
> > > > > > > > > the
> > > > > > > > > > > former
> > > > > > > > > > > > > since remote segments without a matching epoch in
> the
> > > > leader
> > > > > > > > > > > (potentially
> > > > > > > > > > > > > due to unclean leader election) also need to be
> > garbage
> > > > > > > collected.
> > > > > > > > > The
> > > > > > > > > > > > > latter seems to be used for the new leader to
> > determine
> > > > the
> > > > > > > last
> > > > > > > > > tiered
> > > > > > > > > > > > > segment.
> > > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> topicPartition)
> > > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it
> > seems
> > > > that
> > > > > > > they
> > > > > > > > > can
> > > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > > > instead
> > > > > > of
> > > > > > > > > Long?
> > > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > > > implementation and
> > > > > > > > > > > others
> > > > > > > > > > > > > don't?
> > > > > > > > > > > > > 5102.4. Could we define
> > RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> > pass
> > > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array
> > than a
> > > > file
> > > > > > > > > since it
> > > > > > > > > > > will
> > > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it
> > needs
> > > > both
> > > > > > > > > > > baseOffset and
> > > > > > > > > > > > > startOffset. For example, deleteRecords() could
> move
> > the
> > > > > > > > > startOffset
> > > > > > > > > > > to the
> > > > > > > > > > > > > middle of a segment. If we copy the full segment to
> > > > remote
> > > > > > > > > storage, the
> > > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > is
> > > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > > 5103.1 Could we define the default value of
> > non-required
> > > > > > > configs
> > > > > > > > > (e.g
> > > > > > > > > > > the
> > > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > > > default
> > > > > > to
> > > > > > > > > > > retention.ms,
> > > > > > > > > > > > > instead of remote.log.retention.minutes. Similarly,
> > it
> > > > seems
> > > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > > > segment.bytes.
> > > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > > description
> > > > > > > says
> > > > > > > > > "used
> > > > > > > > > > > in
> > > > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > > > indexes
> > > > > > and
> > > > > > > > > clean
> > > > > > > > > > > up
> > > > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > > > config remote.log.reader.threads for fetching
> remote
> > > > data.
> > > > > > It's
> > > > > > > > > weird
> > > > > > > > > > > to
> > > > > > > > > > > > > fetch remote index and log in different thread
> pools
> > > > since
> > > > > > > both are
> > > > > > > > > > > used
> > > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is
> that
> > the
> > > > > > > amount of
> > > > > > > > > > > time to
> > > > > > > > > > > > > back off when there is no work to do? If so,
> perhaps
> > it
> > > > can
> > > > > > be
> > > > > > > > > renamed
> > > > > > > > > > > as
> > > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > rlm_retry_interval_ms
> > > > > > > > > configs?
> > > > > > > > > > > If
> > > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU)
> > of the
> > > > > > index
> > > > > > > > > files
> > > > > > > > > > > of
> > > > > > > > > > > > > remote log segments to avoid multiple index fetches
> > from
> > > > the
> > > > > > > remote
> > > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on
> > disk,
> > > > where
> > > > > > > is it
> > > > > > > > > > > stored?
> > > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> > Earliest
> > > > Local
> > > > > > > > > Offset in
> > > > > > > > > > > > > different places. It would be useful to standardize
> > the
> > > > > > > > > terminology.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux
> > state". It
> > > > > > > listed two
> > > > > > > > > > > options
> > > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5107. Follower to leader transition: It has step 2,
> > but
> > > > not
> > > > > > > step 1.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5108. If a consumer fetches from the remote data
> and
> > the
> > > > > > remote
> > > > > > > > > > > storage is
> > > > > > > > > > > > > not available, what error code is used in the fetch
> > > > response?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns
> > the
> > > > first
> > > > > > > > > message
> > > > > > > > > > > > > offset whose timestamp is >= to the given timestamp
> > in
> > > > the
> > > > > > > request.
> > > > > > > > > > > That
> > > > > > > > > > > > > means it checks in remote log time indexes first,
> > after
> > > > which
> > > > > > > > > local log
> > > > > > > > > > > > > time indexes are checked." Could you document which
> > > > method in
> > > > > > > RLMM
> > > > > > > > > is
> > > > > > > > > > > used
> > > > > > > > > > > > > for this?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> > segment
> > > > > > > metadata of
> > > > > > > > > that
> > > > > > > > > > > > > partition with a delete marker and publishes them
> to
> > > > RLMM."
> > > > > > > This
> > > > > > > > > seems
> > > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for
> > the
> > > > > > > earliest
> > > > > > > > > leader
> > > > > > > > > > > > > epoch by calling
> > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > topicPartition,
> > > > > > > > > > > int
> > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > offset."
> > > > Do we
> > > > > > > need
> > > > > > > > > that
> > > > > > > > > > > > > since replication propagates logStartOffset
> already?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > > > fetching
> > > > > > > from
> > > > > > > > > remote
> > > > > > > > > > > > > storage?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5113. "Committed offsets can be stored in a local
> > file to
> > > > > > avoid
> > > > > > > > > > > reading the
> > > > > > > > > > > > > messages again when a broker is restarted." Could
> you
> > > > > > describe
> > > > > > > the
> > > > > > > > > > > format
> > > > > > > > > > > > > and the location of the file? Also, could the same
> > > > message be
> > > > > > > > > > > processed by
> > > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> > handle
> > > > > > that?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > > 5114.1 There are two records named
> > > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > with
> > > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > document
> > > > > > > whether
> > > > > > > > > > > endOffset
> > > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> > explain
> > > > > > > > > LeaderEpoch a
> > > > > > > > > > > bit
> > > > > > > > > > > > > more? Is that the epoch of the leader when it
> copies
> > the
> > > > > > > segment to
> > > > > > > > > > > remote
> > > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> > more?
> > > > > > Each
> > > > > > > > > record
> > > > > > > > > > > in
> > > > > > > > > > > > > Kafka already has a timestamp field. Could we just
> > use
> > > > that?
> > > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be
> int32?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> > with
> > > > the
> > > > > > > log
> > > > > > > > > > > cleaner
> > > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth
> like
> > > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > > > processes
> > > > > > > it
> > > > > > > > > if it
> > > > > > > > > > > is
> > > > > > > > > > > > > not yet processed earlier." How does it know
> whether
> > > > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5117. Should we add a new MessageFormatter to read
> > the
> > > > tier
> > > > > > > > > metadata
> > > > > > > > > > > topic?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task
> > queue
> > > > size.
> > > > > > > If
> > > > > > > > > the
> > > > > > > > > > > task
> > > > > > > > > > > > > queue is full, broker will stop reading remote log
> > > > segments."
> > > > > > > What
> > > > > > > > > do
> > > > > > > > > > > we
> > > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5119. It would be useful to list all things not
> > > > supported in
> > > > > > > the
> > > > > > > > > first
> > > > > > > > > > > > > version in a Future work or Limitations section.
> For
> > > > example,
> > > > > > > > > compacted
> > > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable
> from
> > > > true to
> > > > > > > false,
> > > > > > > > > > > etc.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > > > > > > kprakasam@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.duggana@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.duggana@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
> <
> > > > > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > > > > (
> > > > > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > > > > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > >
> >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Jun,
Thanks for your reply.  Most of these were discussed in our earlier
syncup, updating here in the mail thread too.

6000. When RSM/RLMM is not available, it seems that we need to return a
retriable error to the affected client requests. So, should we add a new
error code for requests like fetch and listOffset?

When RSM/RLMM is not available, the existing KafkaStorageException is
used, which is a retriable error. It can be considered a storage error
as we could not get the data needed from remote storage or RLMM backed
storage.

9111. RemotePartitionRemover:
91111.1 Could we make it clear that it's part of the default RLMM
implementation and a separate implementation is needed if one has a
customized RLMM?

RemotePartitionRemover is part of default implementation of RLMM.
Custom RLMM implementors need to have a separate implementation of
deleting remote log segments when they receive delete partition state
events.

91111.2 "RPRM gets all the remote log segments for the partition using RLMM
and each of these remote log segments is deleted with the next steps.RLMM
subscribes to the local remote log metadata partitions and it will have the
segment metadata of all the user topic partitions associated with that
remote log metadata partition." It seems that RLMM needs to subscribe to
the remote log metadata partitions first before those segments can be
Deleted?

Yes, RLMM subscribes to local remote log metadata topic partitions.
RemotePartionRemover(RPRM) can get all the remote log segments from
RLMM when it needs to process delete partition marked events.

9111.3 There are still references to "remote log cleaners". They need to be
replaced with RemotePartitionRemover.

Nice catch, replaced those 2 occurrences.

9114.1 Could we add the requirement on log.message.format before enabling
tiered storage?

Updated the KIP.

9116. RemoteLogMetadataFormatter: This is used with the ConsoleConsumer
tool, right? Then, are those new options for ConsoleConsumer and how are
they passed in?

Right, these are for console consumer. It allows taking properties
through `property` flag and user needs to pass them while running
console consumer.


~Satish.

On Tue, 26 Jan 2021 at 01:12, Jun Rao <ju...@confluent.io> wrote:
>
> Hi, Satish,
>
> Thanks for the reply. A few more comments below.
>
> 6000. When RSM/RLMM is not available, it seems that we need to return a
> retriable error to the affected client requests. So, should we add a new
> error code for requests like fetch and listOffset?
>
> 9111. RemotePartitionRemover:
> 91111.1 Could we make it clear that it's part of the default RLMM
> implementation and a separate implementation is needed if one has a
> customized RLMM?
> 91111.2 "RPRM gets all the remote log segments for the partition using RLMM
> and each of these remote log segments is deleted with the next steps.RLMM
> subscribes to the local remote log metadata partitions and it will have the
> segment metadata of all the user topic partitions associated with that
> remote log metadata partition." It seems that RLMM needs to subscribe to
> the remote log metadata partitions first before those segments can be
> deleted?
> 9111.3 There are still references to "remote log cleaners". They need to be
> replaced with RemotePartitionRemover.
>
> 9114.1 Could we add the requirement on log.message.format before enabling
> tiered storage?
>
> 9116. RemoteLogMetadataFormatter: This is used with the ConsoleConsumer
> tool, right? Then, are those new options for ConsoleConsumer and how are
> they passed in?
>
> Thanks,
>
> Jun
>
>
> On Mon, Jan 25, 2021 at 8:02 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi Jun,
> >
> > Thanks for your comments. Please find the inline replies below.
> > Several of these were discussed/clarified in our last discussion.
> >
> > 6000. Since we are returning new error codes, we need to bump up the
> > protocol version for Fetch request. Also, it will be useful to document all
> > new error codes and whether they are retriable or not.
> > This previous comment doesn't seem to be addressed.
> >
> > Updated KIP on fetch protocol and error code.
> >
> > 9110. flat_file_format: Could you define the serialized representation for
> > each field in the header?
> >
> > Added the serialized representation types in the KIP.
> >
> > 9111. RPM has the following 2 steps.
> > "1. The controller publishes delete_partition_marked event to say that the
> > partition is marked for deletion. There can be multiple events published
> > when the controller restarts or failover and this event will be
> > deduplicated by RPM.
> > 2. RPM receives the delete_partition_marked and processes it if it is not
> > yet processed earlier."
> > What triggers RPM to read __remote_log_metadata? Is RPM part of the default
> > RLMM implementation or is it meant for any RLMM implementation?
> >
> > RPM(or RPRM) is only for the default RLMM implementation. RPM receives
> > the delete_partition_marked events from RLMM and acts on them. Updated
> > KIP with more details.
> >
> > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to make
> > it clear in the comment that this is for the initial retry backoff.
> >
> > Updated the KIP.
> >
> > 9113. RLMM:
> > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be updated.
> >
> > Updated javadoc of this method.
> >
> > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> > since all other update events have leaderEpoch?
> >
> > +1 to have that,  updated the KIP.
> >
> > 9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
> > to make it clear this is for deletion?
> >
> > Sure, updated the KIP.
> >
> > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
> > clear this is for segment?
> >
> > Sure, Updated the KIP.
> >
> > 9114.Upgrade:
> > 9114.1 It seems that we require message format > 0.11 for turning on the
> > remote store feature.
> >
> > 9114.2 It's not clear to me why remote.log.storage.system.enable needs to
> > be set to true before bumping up inter.broker.protocol.version.
> >
> > Agree that based on our discussion, this is not required. Upgrade
> > notes is updated.
> >
> > 9114.3 "If the topic-id is not received in the LeaderAndIsr request then
> > remote log storage will not start. But it will log an error message in the
> > log. One way to address this is to do a rolling restart of that broker, so
> > that the leader will be moved to another broker and the controller will
> > send LeaderAndIsr with the registered topic-id." Why is this needed? With
> > KIP-516, after upgrading to the latest protocol, topicIds are auto created.
> >
> > I thought there may be edge cases of not receiving topic-id but you
> > clarified in the meeting that won’t be the case. I agree that it is
> > not needed.
> >
> > 9115. testing: Could you comment on how we plan to do integration and
> > system tests? Do we plan to include, for example, an in-memory
> > implementation of RSM?
> >
> > For integration tests, we can have file based RSM that we have. For
> > system tests, we can have a single node HDFS cluster in one of the
> > containers and use HDFS RSM implementation.
> >
> > 9116. There is still a todo for the message formatter.
> >
> > Updated the KIP with the format.
> >
> >
> > Satish.
> >
> >
> > On Tue, 12 Jan 2021 at 07:07, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Hi, Satish,
> > >
> > > Thanks for the reply. A few more followup comments.
> > >
> > > 6000. Since we are returning new error codes, we need to bump up the
> > > protocol version for Fetch request. Also, it will be useful to document
> > all
> > > new error codes and whether they are retriable or not.
> > > This previous comment doesn't seem to be addressed.
> > >
> > > 9110. flat_file_format: Could you define the serialized representation
> > for
> > > each field in the header?
> > >
> > > 9111. RPM has the following 2 steps.
> > > "1. The controller publishes delete_partition_marked event to say that
> > the
> > > partition is marked for deletion. There can be multiple events published
> > > when the controller restarts or failover and this event will be
> > > deduplicated by RPM.
> > > 2. RPM receives the delete_partition_marked and processes it if it is not
> > > yet processed earlier."
> > > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> > default
> > > RLMM implementation or is it meant for any RLMM implementation?
> > >
> > > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to
> > make
> > > it clear in the comment that this is for the initial retry backoff.
> > >
> > > 9113. RLMM:
> > > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> > updated.
> > > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> > > since all other update events have leaderEpoch?
> > > 9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
> > > to make it clear this is for deletion?
> > > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
> > > clear this is for segment?
> > >
> > > 9114.Upgrade:
> > > 9114.1 It seems that we require message format > 0.11 for turning on the
> > > remote store feature.
> > > 9114.2 It's not clear to me why remote.log.storage.system.enable needs to
> > > be set to true before bumping up inter.broker.protocol.version.
> > > 9114.3 "If the topic-id is not received in the LeaderAndIsr request then
> > > remote log storage will not start. But it will log an error message in
> > the
> > > log. One way to address this is to do a rolling restart of that broker,
> > so
> > > that the leader will be moved to another broker and the controller will
> > > send LeaderAndIsr with the registered topic-id." Why is this needed? With
> > > KIP-516, after upgrading to the latest protocol, topicIds are auto
> > created.
> > >
> > > 9115. testing: Could you comment on how we plan to do integration and
> > > system tests? Do we plan to include, for example, an in-memory
> > > implementation of RSM?
> > >
> > > 9116. There is still a todo for the message formatter.
> > >
> > > Jun
> > >
> > > On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <sa...@gmail.com>
> > > wrote:
> > >
> > > > Hi Jun,
> > > > Thanks for your comments. Please find the inline replies below.
> > > >
> > > > 6022. For packages used for server plugins, the convention is to
> > > > use org.apache.kafka.server. See java-based Authorizer as an example.
> > > >
> > > > Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> > > > ‘org.apache.kafka.server.log.remote.storage’.  Updated in the KIP.
> > > >
> > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The
> > flat
> > > > file captures the state of the remote segments. After a partition is
> > > > deleted, it seems that we just need to remove the partitions's remote
> > > > segments from the flat file.
> > > >
> > > > DeletePartitionState might not yet have been processed by RPM and not
> > > > completed. We  will not have  that in flat  file format  once it
> > > > reaches DELETE_PARTITION_FINISHED state.
> > > >
> > > > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > > > version. It seems that's doable. One can just do the normal upgrade
> > first
> > > > and wait enough time (for producer snapshots to be built), and then
> > enable
> > > > remote storage.
> > > >
> > > > Upgrade notes updates in the KIP.
> > > >
> > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > > > tracking the remote segments when RLMM.onPartitionLeadershipChanges()
> > is
> > > > called with the broker being the leader for __remote_log_metadata
> > > > partition? If so, could we document it?
> > > >
> > > > The current plan is to have that as part of RLMM and RPM uses that to
> > > > get the remote segments list. I will add this detail in the respective
> > > > sections.
> > > >
> > > > Satish.
> > > >
> > > > On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the reply. A few more followup comments.
> > > > >
> > > > > 6022. For packages used for server plugins, the convention is to
> > > > > use org.apache.kafka.server. See java-based Authorizer as an example.
> > > > >
> > > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The
> > flat
> > > > > file captures the state of the remote segments. After a partition is
> > > > > deleted, it seems that we just need to remove the partitions's remote
> > > > > segments from the flat file.
> > > > >
> > > > > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > > > > version. It seems that's doable. One can just do the normal upgrade
> > first
> > > > > and wait enough time (for producer snapshots to be built), and then
> > > > enable
> > > > > remote storage.
> > > > >
> > > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > > > > tracking the remote segments when
> > RLMM.onPartitionLeadershipChanges() is
> > > > > called with the broker being the leader for __remote_log_metadata
> > > > > partition? If so, could we document it?
> > > > >
> > > > > Jun
> > > > >
> > > > > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <
> > kprakasam@confluent.io
> > > > >
> > > > > wrote:
> > > > >
> > > > > > Hi Satish,
> > > > > >
> > > > > > Thanks for the updates! A few more comments below.
> > > > > >
> > > > > > 9001. Under the "Upgrade" section, there is a line mentioning:
> > > > "Upgrade the
> > > > > > existing Kafka cluster to 2.7 version and allow this to run for
> > the log
> > > > > > retention of user topics that you want to enable tiered storage.
> > This
> > > > will
> > > > > > allow all the topics to have the producer snapshots generated for
> > each
> > > > log
> > > > > > segment." -- Which associated change in AK were you referring to
> > here?
> > > > Is
> > > > > > it: https://github.com/apache/kafka/pull/7929 ? It seems like I
> > don't
> > > > see
> > > > > > it in the 2.7 release branch yet, here is the link:
> > > > > > https://github.com/apache/kafka/commits/2.7.
> > > > > >
> > > > > > 9002. Under the "Upgrade" section, the configuration mentioned is
> > > > > > 'remote.log.storage.system.enable'. However, under "Public
> > Interfaces"
> > > > > > section the corresponding configuration is
> > > > 'remote.storage.system.enable'.
> > > > > > Could we use the same one in both, maybe
> > > > > > 'remote.log.storage.system.enable'?
> > > > > >
> > > > > > 9003. Under "Per Topic Configuration", the KIP recommends setting
> > > > > > 'remote.log.storage.enable' to true at a per-topic level. It will
> > be
> > > > useful
> > > > > > to add a line that if the user wants to enable it for all topics,
> > then
> > > > they
> > > > > > should be able to set the cluster-wide default to true. Also, it
> > will
> > > > be
> > > > > > useful to mention that the KIP currently does not support setting
> > it to
> > > > > > false (after it is set to true), and add that to the future work
> > > > section.
> > > > > >
> > > > > > 9004. Under "Committed offsets file format", the sample provided
> > shows
> > > > > > partition number and offset. Is the topic name required for
> > identifying
> > > > > > which topic the partitions belong to?
> > > > > >
> > > > > > 9005. Under "Internal flat-file store format of remote log
> > metadata",
> > > > it
> > > > > > seems useful to specify both topic name and topic ID for debugging
> > > > > > purposes.
> > > > > >
> > > > > > 9006. Under "Internal flat-file store format of remote log
> > metadata",
> > > > the
> > > > > > description of "metadata-topic-offset" currently says "offset of
> > the
> > > > remote
> > > > > > log metadata topic from which this topic partition's remote log
> > > > metadata is
> > > > > > fetched." Just for the wording, perhaps you meant to refer to the
> > > > offset
> > > > > > upto which the file has been committed? i.e. "offset of the remote
> > log
> > > > > > metadata topic upto which this topic partition's remote log
> > metadata
> > > > has
> > > > > > been committed into this file."
> > > > > >
> > > > > > 9007. Under "Internal flat-file store format of remote log
> > metadata",
> > > > the
> > > > > > schema of the payload (i.e. beyond the header) seems to contain the
> > > > events
> > > > > > from the metadata topic. It seems useful to instead persist the
> > > > > > representation of the materialized state of the events, so that
> > for the
> > > > > > same segment only the latest state is stored. Besides reducing
> > storage
> > > > > > footprint, this also is likely to relate directly with the
> > in-memory
> > > > > > representation of the RLMM cache (which probably is some kind of a
> > Map
> > > > with
> > > > > > key being segment ID and value being the segment state), so
> > recovery
> > > > from
> > > > > > disk will be straightforward.
> > > > > >
> > > > > > 9008. Under "Topic deletion lifecycle", step (1), it will be
> > useful to
> > > > > > mention when in the deletion flow does the controller publish the
> > > > > > delete_partition_marked event to say that the partition is marked
> > for
> > > > > > deletion?
> > > > > >
> > > > > > 9009. There are ~4 TODOs in the KIP. Could you please address
> > these or
> > > > > > remove them?
> > > > > >
> > > > > > 9010. There is a reference to a Google doc on the KIP which was
> > used
> > > > > > earlier for discussions. Please could you remove the reference,
> > since
> > > > the
> > > > > > KIP is the source of the truth?
> > > > > >
> > > > > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > > > > >
> > > > > >
> > > > > > Cheers,
> > > > > > Kowshik
> > > > > >
> > > > > >
> > > > > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Jun,
> > > > > > > Thanks for your comments. Please go through the inline replies.
> > > > > > >
> > > > > > >
> > > > > > > 5102.2: It seems that both positions can just be int. Another
> > option
> > > > is
> > > > > > to
> > > > > > > have two methods. Would it be clearer?
> > > > > > >
> > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata,  int startPosition)
> > > > > > throwsRemoteStorageException;
> > > > > > >
> > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata, int startPosition, int endPosition)
> > throws
> > > > > > > RemoteStorageException;
> > > > > > >
> > > > > > > That makes sense to me, updated the KIP.
> > > > > > >
> > > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > log.retention.bytes
> > > > > > > to be consistent with local.log.retention.ms?
> > > > > > >
> > > > > > > Yes, it can be defaulted to log.retention.bytes.
> > > > > > >
> > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > >
> > > > > > > Added TopicIdPartition in the KIP.
> > > > > > >
> > > > > > > 6022: For all public facing classes, could you specify the
> > package
> > > > name?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the reply. A few more comments below.
> > > > > > > >
> > > > > > > > 5102.2: It seems that both positions can just be int. Another
> > > > option is
> > > > > > > to
> > > > > > > > have two methods. Would it be clearer?
> > > > > > > >
> > > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata,
> > > > > > > >                                     int startPosition) throws
> > > > > > > > RemoteStorageException;
> > > > > > > >
> > > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata,
> > > > > > > >                                     int startPosition, int
> > > > endPosition)
> > > > > > > > throws RemoteStorageException;
> > > > > > > >
> > > > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > > > >
> > > > > > > > 6010: What kind of tiering throughput have you seen with 5
> > threads?
> > > > > > > >
> > > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > > log.retention.bytes
> > > > > > > > to be consistent with local.log.retention.ms?
> > > > > > > >
> > > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > > >
> > > > > > > > 6022: For all public facing classes, could you specify the
> > package
> > > > > > name?
> > > > > > > >
> > > > > > > > It seems that you already added the topicId support. Two other
> > > > > > remaining
> > > > > > > > items are (a) the format of local tier metadata storage and (b)
> > > > > > upgrade.
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > > > > satish.duggana@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Jun,
> > > > > > > > > Thanks for your comments. Please find the inline replies
> > below.
> > > > > > > > >
> > > > > > > > > >605.2 It's rare for the follower to need the remote data.
> > So,
> > > > the
> > > > > > > current
> > > > > > > > > approach is fine too. Could you document the process of
> > > > rebuilding
> > > > > > the
> > > > > > > > > producer state since we can't simply trim the producerState
> > to an
> > > > > > > offset in
> > > > > > > > > the middle of a segment.
> > > > > > > > >
> > > > > > > > > Will clarify in the KIP.
> > > > > > > > >
> > > > > > > > > >5102.2 Would it be clearer to make startPosiont long and
> > > > endPosition
> > > > > > > of
> > > > > > > > > Optional<Long>?
> > > > > > > > >
> > > > > > > > > We will have arg checks with respective validation. It is
> > not a
> > > > good
> > > > > > > > > practice to have arguments with optional as mentioned here.
> > > > > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as File
> > > > instead of
> > > > > > > > > ByteBuffer.
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >5102.7 Could you define all public methods for
> > LogSegmentData?
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >5103.5 Could you change the reference to
> > > > rlm_process_interval_ms and
> > > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> > retry
> > > > > > interval
> > > > > > > > > config seems still missing. It would be useful to support
> > > > exponential
> > > > > > > > > backoff with the retry interval config.
> > > > > > > > >
> > > > > > > > > Good point. We wanted the retry with truncated exponential
> > > > backoff,
> > > > > > > > > updated the KIP.
> > > > > > > > >
> > > > > > > > > >5111. "RLM follower fetches the earliest offset for the
> > earliest
> > > > > > > leader
> > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > topicPartition,
> > > > > > > int
> > > > > > > > > leaderEpoch) and updates that as the log start offset." This
> > > > text is
> > > > > > > still
> > > > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >5115. There are still references to "remote log cleaners".
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >6000. Since we are returning new error codes, we need to
> > bump
> > > > up the
> > > > > > > > > protocol version for Fetch request. Also, it will be useful
> > to
> > > > > > > document all
> > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > >
> > > > > > > > > Sure, we will add that in the KIP.
> > > > > > > > >
> > > > > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs():
> > Currently,
> > > > > > > leaderEpoch
> > > > > > > > > is int32 instead of long.
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed
> > > > given
> > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > >
> > > > > > > > > No, it is fixed.
> > > > > > > > >
> > > > > > > > > >6003. RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > topicPartition, long offset, int epochForOffset): Should this
> > > > return
> > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > >
> > > > > > > > > That makes sense, updated.
> > > > > > > > >
> > > > > > > > > >6005. RemoteLogState: It seems it's better to split it
> > between
> > > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> > since
> > > > the
> > > > > > > states
> > > > > > > > > are never shared between the two use cases.
> > > > > > > > >
> > > > > > > > > Agree with that, updated.
> > > > > > > > >
> > > > > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > > > However,
> > > > > > > is it
> > > > > > > > > ture that other than the metadata topic, RLMM just needs to
> > know
> > > > > > > whether
> > > > > > > > > there is a replica assigned to this broker and doesn't need
> > to
> > > > know
> > > > > > > whether
> > > > > > > > > the replica is the leader or the follower?
> > > > > > > > >
> > > > > > > > > That may be true. If the implementation does not need that,
> > it
> > > > can
> > > > > > > > > ignore the information in the callback.
> > > > > > > > >
> > > > > > > > > >6007: "Handle expired remote segments (leader and
> > follower)":
> > > > Why is
> > > > > > > this
> > > > > > > > > needed in both the leader and the follower?
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > > > > >                 "type": "int64",
> > > > > > > > > The segment size can just be int32.
> > > > > > > > >
> > > > > > > > > Updated.
> > > > > > > > >
> > > > > > > > > >6009. For the record format in the log, it seems that we
> > need
> > > > to add
> > > > > > > > > record
> > > > > > > > > type and record version before the serialized bytes. We can
> > > > follow
> > > > > > the
> > > > > > > > > convention used in
> > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > >
> > > > > > > > > Yes, KIP already mentions that these are serialized before
> > the
> > > > > > payload
> > > > > > > > > as below. We will mention explicitly that these two are
> > written
> > > > > > before
> > > > > > > > > the data is written.
> > > > > > > > >
> > > > > > > > > RLMM instance on broker publishes the message to the topic
> > with
> > > > key
> > > > > > as
> > > > > > > > > null and value with the below format.
> > > > > > > > >
> > > > > > > > > type      : unsigned var int, represents the value type. This
> > > > value
> > > > > > is
> > > > > > > > > 'apikey' as mentioned in the schema.
> > > > > > > > > version : unsigned var int, the 'version' number of the type
> > as
> > > > > > > > > mentioned in the schema.
> > > > > > > > > data      : record payload in kafka protocol message format.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > >6010. remote.log.manager.thread.pool.size: The default
> > value is
> > > > 10.
> > > > > > > This
> > > > > > > > > might be too high when enabling the tiered feature for the
> > first
> > > > > > time.
> > > > > > > > > Since there are lots of segments that need to be tiered
> > > > initially, a
> > > > > > > large
> > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > >
> > > > > > > > > Is the default value 5 reasonable?
> > > > > > > > >
> > > > > > > > > 6011. "The number of milli seconds to keep the local log
> > segment
> > > > > > > before it
> > > > > > > > > gets deleted. If not set, the value in
> > `log.retention.minutes` is
> > > > > > > used. If
> > > > > > > > > set to -1, no time limit is applied." We should use
> > > > log.retention.ms
> > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > Nice typo catch. Updated the KIP.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Satish.
> > > > > > > > >
> > > > > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > > > >
> > > > > > > > > > Hi, Satish,
> > > > > > > > > >
> > > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > > >
> > > > > > > > > > 605.2 It's rare for the follower to need the remote data.
> > So,
> > > > the
> > > > > > > current
> > > > > > > > > > approach is fine too. Could you document the process of
> > > > rebuilding
> > > > > > > the
> > > > > > > > > > producer state since we can't simply trim the
> > producerState to
> > > > an
> > > > > > > offset
> > > > > > > > > in
> > > > > > > > > > the middle of a segment.
> > > > > > > > > >
> > > > > > > > > > 5102.2 Would it be clearer to make startPosiont long and
> > > > > > endPosition
> > > > > > > of
> > > > > > > > > > Optional<Long>?
> > > > > > > > > >
> > > > > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File
> > > > instead of
> > > > > > > > > > ByteBuffer.
> > > > > > > > > >
> > > > > > > > > > 5102.7 Could you define all public methods for
> > LogSegmentData?
> > > > > > > > > >
> > > > > > > > > > 5103.5 Could you change the reference to
> > > > rlm_process_interval_ms
> > > > > > and
> > > > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> > retry
> > > > > > > interval
> > > > > > > > > > config seems still missing. It would be useful to support
> > > > > > exponential
> > > > > > > > > > backoff with the retry interval config.
> > > > > > > > > >
> > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > earliest
> > > > > > > leader
> > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > > int
> > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> > This
> > > > text
> > > > > > is
> > > > > > > > > still
> > > > > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > > > > >
> > > > > > > > > > 5115. There are still references to "remote log cleaners".
> > > > > > > > > >
> > > > > > > > > > 6000. Since we are returning new error codes, we need to
> > bump
> > > > up
> > > > > > the
> > > > > > > > > > protocol version for Fetch request. Also, it will be
> > useful to
> > > > > > > document
> > > > > > > > > all
> > > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > > >
> > > > > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs():
> > Currently,
> > > > > > > > > leaderEpoch
> > > > > > > > > > is int32 instead of long.
> > > > > > > > > >
> > > > > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> > needed
> > > > given
> > > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > > >
> > > > > > > > > > 6003. RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > topicPartition, long offset, int epochForOffset): Should
> > this
> > > > > > return
> > > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > > >
> > > > > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to
> > > > pick a
> > > > > > > more
> > > > > > > > > > indicative name so that people understand what epoch this
> > is.
> > > > > > > > > >
> > > > > > > > > > 6005. RemoteLogState: It seems it's better to split it
> > between
> > > > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> > since
> > > > the
> > > > > > > states
> > > > > > > > > > are never shared between the two use cases.
> > > > > > > > > >
> > > > > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > > > However,
> > > > > > > is it
> > > > > > > > > > ture that other than the metadata topic, RLMM just needs to
> > > > know
> > > > > > > whether
> > > > > > > > > > there is a replica assigned to this broker and doesn't
> > need to
> > > > know
> > > > > > > > > whether
> > > > > > > > > > the replica is the leader or the follower?
> > > > > > > > > >
> > > > > > > > > > 6007: "Handle expired remote segments (leader and
> > follower)":
> > > > Why
> > > > > > is
> > > > > > > this
> > > > > > > > > > needed in both the leader and the follower?
> > > > > > > > > >
> > > > > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > > > > >                 "type": "int64",
> > > > > > > > > > The segment size can just be int32.
> > > > > > > > > >
> > > > > > > > > > 6009. For the record format in the log, it seems that we
> > need
> > > > to
> > > > > > add
> > > > > > > > > record
> > > > > > > > > > type and record version before the serialized bytes. We can
> > > > follow
> > > > > > > the
> > > > > > > > > > convention used in
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > > .
> > > > > > > > > >
> > > > > > > > > > 6010. remote.log.manager.thread.pool.size: The default
> > value
> > > > is 10.
> > > > > > > This
> > > > > > > > > > might be too high when enabling the tiered feature for the
> > > > first
> > > > > > > time.
> > > > > > > > > > Since there are lots of segments that need to be tiered
> > > > initially,
> > > > > > a
> > > > > > > > > large
> > > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > > >
> > > > > > > > > > 6011. "The number of milli seconds to keep the local log
> > > > segment
> > > > > > > before
> > > > > > > > > it
> > > > > > > > > > gets deleted. If not set, the value in
> > `log.retention.minutes`
> > > > is
> > > > > > > used.
> > > > > > > > > If
> > > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > > log.retention.ms
> > > > > > > > > > instead of log.retention.minutes.
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi,
> > > > > > > > > > > We updated the KIP with the points mentioned in the
> > earlier
> > > > mail
> > > > > > > > > > > except for KIP-516 related changes. You can go through
> > them
> > > > and
> > > > > > > let us
> > > > > > > > > > > know if you have any comments. We will update the KIP
> > with
> > > > the
> > > > > > > > > > > remaining todo items and KIP-516 related changes by end
> > of
> > > > this
> > > > > > > > > > > week(5th Dec).
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Satish.
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > Thanks for your comments. Please find the inline
> > replies
> > > > below.
> > > > > > > > > > > >
> > > > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting
> > the
> > > > leader
> > > > > > > epoch
> > > > > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > > > > > mentioned an
> > > > > > > > > > > issue
> > > > > > > > > > > > earlier. Suppose the leader's local start offset is
> > 100.
> > > > The
> > > > > > > follower
> > > > > > > > > > > finds
> > > > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > > > producerState
> > > > > > > > > with
> > > > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > > > producerState to
> > > > > > > > > > > > offset 100 requires more work since one needs to
> > download
> > > > the
> > > > > > > > > previous
> > > > > > > > > > > > producerState up to offset 80 and then replay the
> > messages
> > > > from
> > > > > > > 80 to
> > > > > > > > > > > 100.
> > > > > > > > > > > > It seems that it's simpler in this case for the
> > follower
> > > > just
> > > > > > to
> > > > > > > > > take the
> > > > > > > > > > > > remote segment as it is and start fetching from offset
> > 120.
> > > > > > > > > > > >
> > > > > > > > > > > > We chose that approach to avoid any edge cases here. It
> > > > may be
> > > > > > > > > > > > possible that the remote log segment that is received
> > may
> > > > not
> > > > > > > have
> > > > > > > > > the
> > > > > > > > > > > > same leader epoch sequence from 100-120 as it contains
> > on
> > > > the
> > > > > > > > > > > > leader(this can happen due to unclean leader). It is
> > safe
> > > > to
> > > > > > > start
> > > > > > > > > > > > from what the leader returns here.Another way is to
> > find
> > > > the
> > > > > > > remote
> > > > > > > > > > > > log segment
> > > > > > > > > > > >
> > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems
> > that
> > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on
> > the
> > > > > > > replicas
> > > > > > > > > for a
> > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > > > > >
> > > > > > > > > > > > RLMM will always receive the callback for the remote
> > log
> > > > > > metadata
> > > > > > > > > > > > topic partitions hosted on the local broker and these
> > will
> > > > be
> > > > > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > implemented
> > > > now.
> > > > > > > Could
> > > > > > > > > you
> > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > >
> > > > > > > > > > > > We mentioned KIP-516 and how it helps. We will update
> > this
> > > > KIP
> > > > > > > with
> > > > > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > > > > >
> > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > > following two
> > > > > > > APIs
> > > > > > > > > are
> > > > > > > > > > > > used. According to the wiki, the former is used for
> > topic
> > > > > > > deletion
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > latter is used for retention. It seems that retention
> > > > should
> > > > > > use
> > > > > > > the
> > > > > > > > > > > former
> > > > > > > > > > > > since remote segments without a matching epoch in the
> > > > leader
> > > > > > > > > (potentially
> > > > > > > > > > > > due to unclean leader election) also need to be garbage
> > > > > > > collected.
> > > > > > > > > The
> > > > > > > > > > > > latter seems to be used for the new leader to
> > determine the
> > > > > > last
> > > > > > > > > tiered
> > > > > > > > > > > > segment.
> > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > >
> > > > > > > > > > > > Right,.that is what we are currently doing. We will
> > update
> > > > the
> > > > > > > > > > > > javadocs and wiki with that. Earlier, we did not want
> > to
> > > > remove
> > > > > > > the
> > > > > > > > > > > > segments which are not matched with leader epochs from
> > the
> > > > > > ladder
> > > > > > > > > > > > partition as they may be used later by a replica which
> > can
> > > > > > > become a
> > > > > > > > > > > > leader (unclean leader election) and refer those
> > segments.
> > > > But
> > > > > > > that
> > > > > > > > > > > > may leak these segments in remote storage until the
> > topic
> > > > > > > lifetime.
> > > > > > > > > We
> > > > > > > > > > > > decided to cleanup the segments with the oldest incase
> > of
> > > > size
> > > > > > > based
> > > > > > > > > > > > retention also.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems
> > > > that
> > > > > > > they can
> > > > > > > > > > > > use RemoteLogSegmentId instead of
> > RemoteLogSegmentMetadata.
> > > > > > > > > > > >
> > > > > > > > > > > > It will be useful to have metadata for RSM to fetch log
> > > > > > segment.
> > > > > > > It
> > > > > > > > > > > > may create location/path using id with other metadata
> > too.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > > > instead of
> > > > > > > Long?
> > > > > > > > > > > >
> > > > > > > > > > > > Wanted to keep endPosition as optional to read till the
> > > > end of
> > > > > > > the
> > > > > > > > > > > > segment and avoid sentinels.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > implementation
> > > > > > > and
> > > > > > > > > > > others
> > > > > > > > > > > > Don't?
> > > > > > > > > > > >
> > > > > > > > > > > > Actually,  RSM will not have any default
> > implementations.
> > > > > > Those 3
> > > > > > > > > > > > methods were made default earlier for tests etc.
> > Updated
> > > > the
> > > > > > > wiki.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, they will be added.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> > pass
> > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than
> > a
> > > > file
> > > > > > > since
> > > > > > > > > it
> > > > > > > > > > > will
> > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > >
> > > > > > > > > > > > Right, this is in plan.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs
> > > > both
> > > > > > > > > baseOffset
> > > > > > > > > > > and
> > > > > > > > > > > > startOffset. For example, deleteRecords() could move
> > the
> > > > > > > startOffset
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > middle of a segment. If we copy the full segment to
> > remote
> > > > > > > storage,
> > > > > > > > > the
> > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > >
> > > > > > > > > > > > Good point. startOffset is baseOffset by default, if
> > not
> > > > set
> > > > > > > > > explicitly.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, updated the wiki.
> > > > > > > > > > > >
> > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > >
> > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > >
> > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > 5103.1 Could we define the default value of
> > non-required
> > > > > > configs
> > > > > > > > > (e.g the
> > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, that makes sense.
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > > > default to
> > > > > > > > > > > retention.ms,
> > > > > > > > > > > > instead of remote.log.retention.minutes. Similarly, it
> > > > seems
> > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > segment.bytes.
> > > > > > > > > > > >
> > > > > > > > > > > > Right, we do not have  remote.log.retention as we
> > discussed
> > > > > > > earlier.
> > > > > > > > > > > > Thanks for catching the typo.
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > description
> > > > > > says
> > > > > > > > > "used in
> > > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > > > indexes and
> > > > > > > > > clean up
> > > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > > config remote.log.reader.threads for fetching remote
> > data.
> > > > It's
> > > > > > > > > weird to
> > > > > > > > > > > > fetch remote index and log in different thread pools
> > since
> > > > both
> > > > > > > are
> > > > > > > > > used
> > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > >
> > > > > > > > > > > > Right, remote.log.manager.thread.pool is mainly used
> > for
> > > > > > > copy/cleanup
> > > > > > > > > > > > activities. Fetch path always goes through
> > > > > > > remote.log.reader.threads.
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that
> > the
> > > > amount
> > > > > > > of
> > > > > > > > > time
> > > > > > > > > > > to
> > > > > > > > > > > > back off when there is no work to do? If so, perhaps it
> > > > can be
> > > > > > > > > renamed as
> > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > >
> > > > > > > > > > > > This is the delay interval for each iteration. It may
> > be
> > > > > > renamed
> > > > > > > to
> > > > > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > > > > >
> > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > rlm_retry_interval_ms
> > > > > > > > > configs? If
> > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > >
> > > > > > > > > > > > remote.log.manager.task.interval.ms is the process
> > > > internal,
> > > > > > > retry
> > > > > > > > > > > > interval is missing in the configs, which will be
> > updated
> > > > in
> > > > > > the
> > > > > > > KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of
> > the
> > > > index
> > > > > > > > > files of
> > > > > > > > > > > > remote log segments to avoid multiple index fetches
> > from
> > > > the
> > > > > > > remote
> > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on disk,
> > > > where is
> > > > > > > it
> > > > > > > > > > > stored?
> > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > >
> > > > > > > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > > > > > > `remote-log-index-cache` under log dir. We plan to
> > have a
> > > > > > config
> > > > > > > for
> > > > > > > > > > > > that instead of default. We will have a configuration
> > for
> > > > that.
> > > > > > > > > > > >
> > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest
> > > > Local
> > > > > > > Offset
> > > > > > > > > in
> > > > > > > > > > > > different places. It would be useful to standardize the
> > > > > > > terminology.
> > > > > > > > > > > >
> > > > > > > > > > > > Sure.
> > > > > > > > > > > >
> > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state".
> > It
> > > > listed
> > > > > > > two
> > > > > > > > > > > options
> > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > > > > > > > >
> > > > > > > > > > > > 5107. Follower to leader transition: It has step 2,
> > but not
> > > > > > step
> > > > > > > 1.
> > > > > > > > > > > > Step-1 is there but it is not explicitly highlighted.
> > It is
> > > > > > > previous
> > > > > > > > > > > > table to step-2.
> > > > > > > > > > > >
> > > > > > > > > > > > 5108. If a consumer fetches from the remote data and
> > the
> > > > remote
> > > > > > > > > storage
> > > > > > > > > > > is
> > > > > > > > > > > > not available, what error code is used in the fetch
> > > > response?
> > > > > > > > > > > >
> > > > > > > > > > > > Good point. We have not yet defined the error for this
> > > > case. We
> > > > > > > need
> > > > > > > > > > > > to define an error message and send the same in fetch
> > > > response.
> > > > > > > > > > > >
> > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the
> > > > first
> > > > > > > message
> > > > > > > > > > > > offset whose timestamp is >= to the given timestamp in
> > the
> > > > > > > request.
> > > > > > > > > That
> > > > > > > > > > > > means it checks in remote log time indexes first, after
> > > > which
> > > > > > > local
> > > > > > > > > log
> > > > > > > > > > > > time indexes are checked." Could you document which
> > method
> > > > in
> > > > > > > RLMM is
> > > > > > > > > > > used
> > > > > > > > > > > > for this?
> > > > > > > > > > > >
> > > > > > > > > > > > Okay.
> > > > > > > > > > > >
> > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > > > metadata
> > > > > > > of
> > > > > > > > > that
> > > > > > > > > > > > partition with a delete marker and publishes them to
> > RLMM."
> > > > > > This
> > > > > > > > > seems
> > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > >
> > > > > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > > > > >
> > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > > > earliest
> > > > > > > > > leader
> > > > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > topicPartition,
> > > > > > > > > > > int
> > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > offset." Do
> > > > we
> > > > > > > need
> > > > > > > > > that
> > > > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > > > >
> > > > > > > > > > > > Good point. Right, existing replication protocol takes
> > > > care of
> > > > > > > > > > > > updating the followers’s log start offset received
> > from the
> > > > > > > leader.
> > > > > > > > > > > >
> > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > fetching
> > > > > > from
> > > > > > > > > remote
> > > > > > > > > > > > storage?
> > > > > > > > > > > >
> > > > > > > > > > > > Remote reads may fail within the current default wait
> > > > time, but
> > > > > > > > > > > > subsequent fetches would be able to serve as that data
> > is
> > > > > > stored
> > > > > > > in
> > > > > > > > > > > > the local cache. This cache is currently implemented in
> > > > RSMs.
> > > > > > > But we
> > > > > > > > > > > > plan to pull this into the remote log messaging layer
> > in
> > > > > > future.
> > > > > > > > > > > >
> > > > > > > > > > > > 5113. "Committed offsets can be stored in a local file
> > to
> > > > avoid
> > > > > > > > > reading
> > > > > > > > > > > the
> > > > > > > > > > > > messages again when a broker is restarted." Could you
> > > > describe
> > > > > > > the
> > > > > > > > > format
> > > > > > > > > > > > and the location of the file? Also, could the same
> > message
> > > > be
> > > > > > > > > processed
> > > > > > > > > > > by
> > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> > handle
> > > > that?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > 5114.1 There are two records named
> > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > with
> > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > >
> > > > > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > > > > >
> > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > document
> > > > > > whether
> > > > > > > > > > > endOffset
> > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > > >
> > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> > explain
> > > > > > > LeaderEpoch
> > > > > > > > > a
> > > > > > > > > > > bit
> > > > > > > > > > > > more? Is that the epoch of the leader when it copies
> > the
> > > > > > segment
> > > > > > > to
> > > > > > > > > > > remote
> > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > >
> > > > > > > > > > > > Right, this is the leader epoch of the broker which
> > copied
> > > > this
> > > > > > > > > > > > segment. This is helpful in reason about which broker
> > > > copied
> > > > > > the
> > > > > > > > > > > > segment to remote storage.
> > > > > > > > > > > >
> > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> > more?
> > > > Each
> > > > > > > > > record in
> > > > > > > > > > > > Kafka already has a timestamp field. Could we just use
> > > > that?
> > > > > > > > > > > >
> > > > > > > > > > > > This is the  timestamp at which  the respective event
> > > > occurred.
> > > > > > > Added
> > > > > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any
> > other
> > > > > > > > > > > > implementation. We thought about that but it looked
> > > > cleaner to
> > > > > > > use at
> > > > > > > > > > > > the message structure level instead of getting that
> > from
> > > > the
> > > > > > > consumer
> > > > > > > > > > > > record and using that to build the respective event.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > > > >
> > > > > > > > > > > > Right, it looks like config allows only int value >=
> > 14.
> > > > > > > > > > > >
> > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> > with
> > > > the
> > > > > > log
> > > > > > > > > cleaner
> > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > >
> > > > > > > > > > > > I am fine with RemotePartitionRemover or
> > > > > > > RemoteLogDeletionManager(we
> > > > > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > > > > >
> > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > > > processes
> > > > > > it
> > > > > > > if
> > > > > > > > > it is
> > > > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > > >
> > > > > > > > > > > > This is to handle duplicate delete_partition_marked
> > > > events. RLC
> > > > > > > > > > > > internally maintains a state for the delete_partition
> > > > events
> > > > > > and
> > > > > > > if
> > > > > > > > > it
> > > > > > > > > > > > already has an existing event then it ignores if it is
> > > > already
> > > > > > > being
> > > > > > > > > > > > processed.
> > > > > > > > > > > >
> > > > > > > > > > > > 5117. Should we add a new MessageFormatter to read the
> > tier
> > > > > > > metadata
> > > > > > > > > > > topic?
> > > > > > > > > > > >
> > > > > > > > > > > > Right, this is in plan but did not mention it in the
> > KIP.
> > > > This
> > > > > > > will
> > > > > > > > > be
> > > > > > > > > > > > useful for debugging purposes too.
> > > > > > > > > > > >
> > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task queue
> > > > size.
> > > > > > If
> > > > > > > the
> > > > > > > > > task
> > > > > > > > > > > > queue is full, broker will stop reading remote log
> > > > segments."
> > > > > > > What
> > > > > > > > > do we
> > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > >
> > > > > > > > > > > > We return an error response for that partition.
> > > > > > > > > > > >
> > > > > > > > > > > > 5119. It would be useful to list all things not
> > supported
> > > > in
> > > > > > the
> > > > > > > > > first
> > > > > > > > > > > > version in a Future work or Limitations section. For
> > > > example,
> > > > > > > > > compacted
> > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> > true
> > > > to
> > > > > > > false,
> > > > > > > > > etc.
> > > > > > > > > > > >
> > > > > > > > > > > > We already have a non-goals section which is filled
> > with
> > > > some
> > > > > > of
> > > > > > > > > these
> > > > > > > > > > > > details. Do we need another limitations section?
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Satish.
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <
> > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > below.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting
> > the
> > > > > > leader
> > > > > > > > > epoch
> > > > > > > > > > > > > sequence received from remote storage to [LSO,
> > ELO]." I
> > > > > > > mentioned
> > > > > > > > > an
> > > > > > > > > > > issue
> > > > > > > > > > > > > earlier. Suppose the leader's local start offset is
> > 100.
> > > > The
> > > > > > > > > follower
> > > > > > > > > > > finds
> > > > > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > > > producerState
> > > > > > > > > > > with
> > > > > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > > > producerState
> > > > > > > > > to
> > > > > > > > > > > > > offset 100 requires more work since one needs to
> > > > download the
> > > > > > > > > previous
> > > > > > > > > > > > > producerState up to offset 80 and then replay the
> > > > messages
> > > > > > > from 80
> > > > > > > > > to
> > > > > > > > > > > 100.
> > > > > > > > > > > > > It seems that it's simpler in this case for the
> > follower
> > > > just
> > > > > > > to
> > > > > > > > > take
> > > > > > > > > > > the
> > > > > > > > > > > > > remote segment as it is and start fetching from
> > offset
> > > > 120.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems
> > that
> > > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called
> > on the
> > > > > > > replicas
> > > > > > > > > for
> > > > > > > > > > > a
> > > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > > obtains the metadata for remote segments for
> > deletion.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> > implemented
> > > > now.
> > > > > > > > > Could you
> > > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > > following
> > > > > > two
> > > > > > > > > APIs
> > > > > > > > > > > are
> > > > > > > > > > > > > used. According to the wiki, the former is used for
> > topic
> > > > > > > deletion
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > > latter is used for retention. It seems that retention
> > > > should
> > > > > > > use
> > > > > > > > > the
> > > > > > > > > > > former
> > > > > > > > > > > > > since remote segments without a matching epoch in the
> > > > leader
> > > > > > > > > > > (potentially
> > > > > > > > > > > > > due to unclean leader election) also need to be
> > garbage
> > > > > > > collected.
> > > > > > > > > The
> > > > > > > > > > > > > latter seems to be used for the new leader to
> > determine
> > > > the
> > > > > > > last
> > > > > > > > > tiered
> > > > > > > > > > > > > segment.
> > > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it
> > seems
> > > > that
> > > > > > > they
> > > > > > > > > can
> > > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > > > instead
> > > > > > of
> > > > > > > > > Long?
> > > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > > > implementation and
> > > > > > > > > > > others
> > > > > > > > > > > > > don't?
> > > > > > > > > > > > > 5102.4. Could we define
> > RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> > pass
> > > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array
> > than a
> > > > file
> > > > > > > > > since it
> > > > > > > > > > > will
> > > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it
> > needs
> > > > both
> > > > > > > > > > > baseOffset and
> > > > > > > > > > > > > startOffset. For example, deleteRecords() could move
> > the
> > > > > > > > > startOffset
> > > > > > > > > > > to the
> > > > > > > > > > > > > middle of a segment. If we copy the full segment to
> > > > remote
> > > > > > > > > storage, the
> > > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > is
> > > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > > 5103.1 Could we define the default value of
> > non-required
> > > > > > > configs
> > > > > > > > > (e.g
> > > > > > > > > > > the
> > > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > > > default
> > > > > > to
> > > > > > > > > > > retention.ms,
> > > > > > > > > > > > > instead of remote.log.retention.minutes. Similarly,
> > it
> > > > seems
> > > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > > > segment.bytes.
> > > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > > description
> > > > > > > says
> > > > > > > > > "used
> > > > > > > > > > > in
> > > > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > > > indexes
> > > > > > and
> > > > > > > > > clean
> > > > > > > > > > > up
> > > > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > > > config remote.log.reader.threads for fetching remote
> > > > data.
> > > > > > It's
> > > > > > > > > weird
> > > > > > > > > > > to
> > > > > > > > > > > > > fetch remote index and log in different thread pools
> > > > since
> > > > > > > both are
> > > > > > > > > > > used
> > > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that
> > the
> > > > > > > amount of
> > > > > > > > > > > time to
> > > > > > > > > > > > > back off when there is no work to do? If so, perhaps
> > it
> > > > can
> > > > > > be
> > > > > > > > > renamed
> > > > > > > > > > > as
> > > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > > rlm_retry_interval_ms
> > > > > > > > > configs?
> > > > > > > > > > > If
> > > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU)
> > of the
> > > > > > index
> > > > > > > > > files
> > > > > > > > > > > of
> > > > > > > > > > > > > remote log segments to avoid multiple index fetches
> > from
> > > > the
> > > > > > > remote
> > > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on
> > disk,
> > > > where
> > > > > > > is it
> > > > > > > > > > > stored?
> > > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> > Earliest
> > > > Local
> > > > > > > > > Offset in
> > > > > > > > > > > > > different places. It would be useful to standardize
> > the
> > > > > > > > > terminology.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux
> > state". It
> > > > > > > listed two
> > > > > > > > > > > options
> > > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5107. Follower to leader transition: It has step 2,
> > but
> > > > not
> > > > > > > step 1.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5108. If a consumer fetches from the remote data and
> > the
> > > > > > remote
> > > > > > > > > > > storage is
> > > > > > > > > > > > > not available, what error code is used in the fetch
> > > > response?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns
> > the
> > > > first
> > > > > > > > > message
> > > > > > > > > > > > > offset whose timestamp is >= to the given timestamp
> > in
> > > > the
> > > > > > > request.
> > > > > > > > > > > That
> > > > > > > > > > > > > means it checks in remote log time indexes first,
> > after
> > > > which
> > > > > > > > > local log
> > > > > > > > > > > > > time indexes are checked." Could you document which
> > > > method in
> > > > > > > RLMM
> > > > > > > > > is
> > > > > > > > > > > used
> > > > > > > > > > > > > for this?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> > segment
> > > > > > > metadata of
> > > > > > > > > that
> > > > > > > > > > > > > partition with a delete marker and publishes them to
> > > > RLMM."
> > > > > > > This
> > > > > > > > > seems
> > > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for
> > the
> > > > > > > earliest
> > > > > > > > > leader
> > > > > > > > > > > > > epoch by calling
> > RLMM.earliestLogOffset(TopicPartition
> > > > > > > > > topicPartition,
> > > > > > > > > > > int
> > > > > > > > > > > > > leaderEpoch) and updates that as the log start
> > offset."
> > > > Do we
> > > > > > > need
> > > > > > > > > that
> > > > > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > > > fetching
> > > > > > > from
> > > > > > > > > remote
> > > > > > > > > > > > > storage?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5113. "Committed offsets can be stored in a local
> > file to
> > > > > > avoid
> > > > > > > > > > > reading the
> > > > > > > > > > > > > messages again when a broker is restarted." Could you
> > > > > > describe
> > > > > > > the
> > > > > > > > > > > format
> > > > > > > > > > > > > and the location of the file? Also, could the same
> > > > message be
> > > > > > > > > > > processed by
> > > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> > handle
> > > > > > that?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > > 5114.1 There are two records named
> > > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > > with
> > > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> > document
> > > > > > > whether
> > > > > > > > > > > endOffset
> > > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> > explain
> > > > > > > > > LeaderEpoch a
> > > > > > > > > > > bit
> > > > > > > > > > > > > more? Is that the epoch of the leader when it copies
> > the
> > > > > > > segment to
> > > > > > > > > > > remote
> > > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> > more?
> > > > > > Each
> > > > > > > > > record
> > > > > > > > > > > in
> > > > > > > > > > > > > Kafka already has a timestamp field. Could we just
> > use
> > > > that?
> > > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> > with
> > > > the
> > > > > > > log
> > > > > > > > > > > cleaner
> > > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > > > processes
> > > > > > > it
> > > > > > > > > if it
> > > > > > > > > > > is
> > > > > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5117. Should we add a new MessageFormatter to read
> > the
> > > > tier
> > > > > > > > > metadata
> > > > > > > > > > > topic?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task
> > queue
> > > > size.
> > > > > > > If
> > > > > > > > > the
> > > > > > > > > > > task
> > > > > > > > > > > > > queue is full, broker will stop reading remote log
> > > > segments."
> > > > > > > What
> > > > > > > > > do
> > > > > > > > > > > we
> > > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5119. It would be useful to list all things not
> > > > supported in
> > > > > > > the
> > > > > > > > > first
> > > > > > > > > > > > > version in a Future work or Limitations section. For
> > > > example,
> > > > > > > > > compacted
> > > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> > > > true to
> > > > > > > false,
> > > > > > > > > > > etc.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > > > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > > > > (
> > > > > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > > > > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > >
> >

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

Posted by Jun Rao <ju...@confluent.io>.
Hi, Satish,

Thanks for the reply. A few more comments below.

6000. When RSM/RLMM is not available, it seems that we need to return a
retriable error to the affected client requests. So, should we add a new
error code for requests like fetch and listOffset?

9111. RemotePartitionRemover:
91111.1 Could we make it clear that it's part of the default RLMM
implementation and a separate implementation is needed if one has a
customized RLMM?
91111.2 "RPRM gets all the remote log segments for the partition using RLMM
and each of these remote log segments is deleted with the next steps.RLMM
subscribes to the local remote log metadata partitions and it will have the
segment metadata of all the user topic partitions associated with that
remote log metadata partition." It seems that RLMM needs to subscribe to
the remote log metadata partitions first before those segments can be
deleted?
9111.3 There are still references to "remote log cleaners". They need to be
replaced with RemotePartitionRemover.

9114.1 Could we add the requirement on log.message.format before enabling
tiered storage?

9116. RemoteLogMetadataFormatter: This is used with the ConsoleConsumer
tool, right? Then, are those new options for ConsoleConsumer and how are
they passed in?

Thanks,

Jun


On Mon, Jan 25, 2021 at 8:02 AM Satish Duggana <sa...@gmail.com>
wrote:

> Hi Jun,
>
> Thanks for your comments. Please find the inline replies below.
> Several of these were discussed/clarified in our last discussion.
>
> 6000. Since we are returning new error codes, we need to bump up the
> protocol version for Fetch request. Also, it will be useful to document all
> new error codes and whether they are retriable or not.
> This previous comment doesn't seem to be addressed.
>
> Updated KIP on fetch protocol and error code.
>
> 9110. flat_file_format: Could you define the serialized representation for
> each field in the header?
>
> Added the serialized representation types in the KIP.
>
> 9111. RPM has the following 2 steps.
> "1. The controller publishes delete_partition_marked event to say that the
> partition is marked for deletion. There can be multiple events published
> when the controller restarts or failover and this event will be
> deduplicated by RPM.
> 2. RPM receives the delete_partition_marked and processes it if it is not
> yet processed earlier."
> What triggers RPM to read __remote_log_metadata? Is RPM part of the default
> RLMM implementation or is it meant for any RLMM implementation?
>
> RPM(or RPRM) is only for the default RLMM implementation. RPM receives
> the delete_partition_marked events from RLMM and acts on them. Updated
> KIP with more details.
>
> 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to make
> it clear in the comment that this is for the initial retry backoff.
>
> Updated the KIP.
>
> 9113. RLMM:
> 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be updated.
>
> Updated javadoc of this method.
>
> 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> since all other update events have leaderEpoch?
>
> +1 to have that,  updated the KIP.
>
> 9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
> to make it clear this is for deletion?
>
> Sure, updated the KIP.
>
> 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
> clear this is for segment?
>
> Sure, Updated the KIP.
>
> 9114.Upgrade:
> 9114.1 It seems that we require message format > 0.11 for turning on the
> remote store feature.
>
> 9114.2 It's not clear to me why remote.log.storage.system.enable needs to
> be set to true before bumping up inter.broker.protocol.version.
>
> Agree that based on our discussion, this is not required. Upgrade
> notes is updated.
>
> 9114.3 "If the topic-id is not received in the LeaderAndIsr request then
> remote log storage will not start. But it will log an error message in the
> log. One way to address this is to do a rolling restart of that broker, so
> that the leader will be moved to another broker and the controller will
> send LeaderAndIsr with the registered topic-id." Why is this needed? With
> KIP-516, after upgrading to the latest protocol, topicIds are auto created.
>
> I thought there may be edge cases of not receiving topic-id but you
> clarified in the meeting that won’t be the case. I agree that it is
> not needed.
>
> 9115. testing: Could you comment on how we plan to do integration and
> system tests? Do we plan to include, for example, an in-memory
> implementation of RSM?
>
> For integration tests, we can have file based RSM that we have. For
> system tests, we can have a single node HDFS cluster in one of the
> containers and use HDFS RSM implementation.
>
> 9116. There is still a todo for the message formatter.
>
> Updated the KIP with the format.
>
>
> Satish.
>
>
> On Tue, 12 Jan 2021 at 07:07, Jun Rao <ju...@confluent.io> wrote:
> >
> > Hi, Satish,
> >
> > Thanks for the reply. A few more followup comments.
> >
> > 6000. Since we are returning new error codes, we need to bump up the
> > protocol version for Fetch request. Also, it will be useful to document
> all
> > new error codes and whether they are retriable or not.
> > This previous comment doesn't seem to be addressed.
> >
> > 9110. flat_file_format: Could you define the serialized representation
> for
> > each field in the header?
> >
> > 9111. RPM has the following 2 steps.
> > "1. The controller publishes delete_partition_marked event to say that
> the
> > partition is marked for deletion. There can be multiple events published
> > when the controller restarts or failover and this event will be
> > deduplicated by RPM.
> > 2. RPM receives the delete_partition_marked and processes it if it is not
> > yet processed earlier."
> > What triggers RPM to read __remote_log_metadata? Is RPM part of the
> default
> > RLMM implementation or is it meant for any RLMM implementation?
> >
> > 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to
> make
> > it clear in the comment that this is for the initial retry backoff.
> >
> > 9113. RLMM:
> > 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be
> updated.
> > 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> > since all other update events have leaderEpoch?
> > 9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
> > to make it clear this is for deletion?
> > 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
> > clear this is for segment?
> >
> > 9114.Upgrade:
> > 9114.1 It seems that we require message format > 0.11 for turning on the
> > remote store feature.
> > 9114.2 It's not clear to me why remote.log.storage.system.enable needs to
> > be set to true before bumping up inter.broker.protocol.version.
> > 9114.3 "If the topic-id is not received in the LeaderAndIsr request then
> > remote log storage will not start. But it will log an error message in
> the
> > log. One way to address this is to do a rolling restart of that broker,
> so
> > that the leader will be moved to another broker and the controller will
> > send LeaderAndIsr with the registered topic-id." Why is this needed? With
> > KIP-516, after upgrading to the latest protocol, topicIds are auto
> created.
> >
> > 9115. testing: Could you comment on how we plan to do integration and
> > system tests? Do we plan to include, for example, an in-memory
> > implementation of RSM?
> >
> > 9116. There is still a todo for the message formatter.
> >
> > Jun
> >
> > On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <sa...@gmail.com>
> > wrote:
> >
> > > Hi Jun,
> > > Thanks for your comments. Please find the inline replies below.
> > >
> > > 6022. For packages used for server plugins, the convention is to
> > > use org.apache.kafka.server. See java-based Authorizer as an example.
> > >
> > > Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> > > ‘org.apache.kafka.server.log.remote.storage’.  Updated in the KIP.
> > >
> > > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The
> flat
> > > file captures the state of the remote segments. After a partition is
> > > deleted, it seems that we just need to remove the partitions's remote
> > > segments from the flat file.
> > >
> > > DeletePartitionState might not yet have been processed by RPM and not
> > > completed. We  will not have  that in flat  file format  once it
> > > reaches DELETE_PARTITION_FINISHED state.
> > >
> > > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > > version. It seems that's doable. One can just do the normal upgrade
> first
> > > and wait enough time (for producer snapshots to be built), and then
> enable
> > > remote storage.
> > >
> > > Upgrade notes updates in the KIP.
> > >
> > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > > tracking the remote segments when RLMM.onPartitionLeadershipChanges()
> is
> > > called with the broker being the leader for __remote_log_metadata
> > > partition? If so, could we document it?
> > >
> > > The current plan is to have that as part of RLMM and RPM uses that to
> > > get the remote segments list. I will add this detail in the respective
> > > sections.
> > >
> > > Satish.
> > >
> > > On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > Hi, Satish,
> > > >
> > > > Thanks for the reply. A few more followup comments.
> > > >
> > > > 6022. For packages used for server plugins, the convention is to
> > > > use org.apache.kafka.server. See java-based Authorizer as an example.
> > > >
> > > > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The
> flat
> > > > file captures the state of the remote segments. After a partition is
> > > > deleted, it seems that we just need to remove the partitions's remote
> > > > segments from the flat file.
> > > >
> > > > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > > > version. It seems that's doable. One can just do the normal upgrade
> first
> > > > and wait enough time (for producer snapshots to be built), and then
> > > enable
> > > > remote storage.
> > > >
> > > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > > > tracking the remote segments when
> RLMM.onPartitionLeadershipChanges() is
> > > > called with the broker being the leader for __remote_log_metadata
> > > > partition? If so, could we document it?
> > > >
> > > > Jun
> > > >
> > > > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <
> kprakasam@confluent.io
> > > >
> > > > wrote:
> > > >
> > > > > Hi Satish,
> > > > >
> > > > > Thanks for the updates! A few more comments below.
> > > > >
> > > > > 9001. Under the "Upgrade" section, there is a line mentioning:
> > > "Upgrade the
> > > > > existing Kafka cluster to 2.7 version and allow this to run for
> the log
> > > > > retention of user topics that you want to enable tiered storage.
> This
> > > will
> > > > > allow all the topics to have the producer snapshots generated for
> each
> > > log
> > > > > segment." -- Which associated change in AK were you referring to
> here?
> > > Is
> > > > > it: https://github.com/apache/kafka/pull/7929 ? It seems like I
> don't
> > > see
> > > > > it in the 2.7 release branch yet, here is the link:
> > > > > https://github.com/apache/kafka/commits/2.7.
> > > > >
> > > > > 9002. Under the "Upgrade" section, the configuration mentioned is
> > > > > 'remote.log.storage.system.enable'. However, under "Public
> Interfaces"
> > > > > section the corresponding configuration is
> > > 'remote.storage.system.enable'.
> > > > > Could we use the same one in both, maybe
> > > > > 'remote.log.storage.system.enable'?
> > > > >
> > > > > 9003. Under "Per Topic Configuration", the KIP recommends setting
> > > > > 'remote.log.storage.enable' to true at a per-topic level. It will
> be
> > > useful
> > > > > to add a line that if the user wants to enable it for all topics,
> then
> > > they
> > > > > should be able to set the cluster-wide default to true. Also, it
> will
> > > be
> > > > > useful to mention that the KIP currently does not support setting
> it to
> > > > > false (after it is set to true), and add that to the future work
> > > section.
> > > > >
> > > > > 9004. Under "Committed offsets file format", the sample provided
> shows
> > > > > partition number and offset. Is the topic name required for
> identifying
> > > > > which topic the partitions belong to?
> > > > >
> > > > > 9005. Under "Internal flat-file store format of remote log
> metadata",
> > > it
> > > > > seems useful to specify both topic name and topic ID for debugging
> > > > > purposes.
> > > > >
> > > > > 9006. Under "Internal flat-file store format of remote log
> metadata",
> > > the
> > > > > description of "metadata-topic-offset" currently says "offset of
> the
> > > remote
> > > > > log metadata topic from which this topic partition's remote log
> > > metadata is
> > > > > fetched." Just for the wording, perhaps you meant to refer to the
> > > offset
> > > > > upto which the file has been committed? i.e. "offset of the remote
> log
> > > > > metadata topic upto which this topic partition's remote log
> metadata
> > > has
> > > > > been committed into this file."
> > > > >
> > > > > 9007. Under "Internal flat-file store format of remote log
> metadata",
> > > the
> > > > > schema of the payload (i.e. beyond the header) seems to contain the
> > > events
> > > > > from the metadata topic. It seems useful to instead persist the
> > > > > representation of the materialized state of the events, so that
> for the
> > > > > same segment only the latest state is stored. Besides reducing
> storage
> > > > > footprint, this also is likely to relate directly with the
> in-memory
> > > > > representation of the RLMM cache (which probably is some kind of a
> Map
> > > with
> > > > > key being segment ID and value being the segment state), so
> recovery
> > > from
> > > > > disk will be straightforward.
> > > > >
> > > > > 9008. Under "Topic deletion lifecycle", step (1), it will be
> useful to
> > > > > mention when in the deletion flow does the controller publish the
> > > > > delete_partition_marked event to say that the partition is marked
> for
> > > > > deletion?
> > > > >
> > > > > 9009. There are ~4 TODOs in the KIP. Could you please address
> these or
> > > > > remove them?
> > > > >
> > > > > 9010. There is a reference to a Google doc on the KIP which was
> used
> > > > > earlier for discussions. Please could you remove the reference,
> since
> > > the
> > > > > KIP is the source of the truth?
> > > > >
> > > > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > > > >
> > > > >
> > > > > Cheers,
> > > > > Kowshik
> > > > >
> > > > >
> > > > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> > > satish.duggana@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > > Thanks for your comments. Please go through the inline replies.
> > > > > >
> > > > > >
> > > > > > 5102.2: It seems that both positions can just be int. Another
> option
> > > is
> > > > > to
> > > > > > have two methods. Would it be clearer?
> > > > > >
> > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata,  int startPosition)
> > > > > throwsRemoteStorageException;
> > > > > >
> > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata, int startPosition, int endPosition)
> throws
> > > > > > RemoteStorageException;
> > > > > >
> > > > > > That makes sense to me, updated the KIP.
> > > > > >
> > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > > 6020: local.log.retention.bytes: Should it default to
> > > log.retention.bytes
> > > > > > to be consistent with local.log.retention.ms?
> > > > > >
> > > > > > Yes, it can be defaulted to log.retention.bytes.
> > > > > >
> > > > > > 6021: Could you define TopicIdPartition?
> > > > > >
> > > > > > Added TopicIdPartition in the KIP.
> > > > > >
> > > > > > 6022: For all public facing classes, could you specify the
> package
> > > name?
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > >
> > > > > > Thanks,
> > > > > > Satish.
> > > > > >
> > > > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > >
> > > > > > > Hi, Satish,
> > > > > > >
> > > > > > > Thanks for the reply. A few more comments below.
> > > > > > >
> > > > > > > 5102.2: It seems that both positions can just be int. Another
> > > option is
> > > > > > to
> > > > > > > have two methods. Would it be clearer?
> > > > > > >
> > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata,
> > > > > > >                                     int startPosition) throws
> > > > > > > RemoteStorageException;
> > > > > > >
> > > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata,
> > > > > > >                                     int startPosition, int
> > > endPosition)
> > > > > > > throws RemoteStorageException;
> > > > > > >
> > > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > > >
> > > > > > > 6010: What kind of tiering throughput have you seen with 5
> threads?
> > > > > > >
> > > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > > log.retention.bytes
> > > > > > > to be consistent with local.log.retention.ms?
> > > > > > >
> > > > > > > 6021: Could you define TopicIdPartition?
> > > > > > >
> > > > > > > 6022: For all public facing classes, could you specify the
> package
> > > > > name?
> > > > > > >
> > > > > > > It seems that you already added the topicId support. Two other
> > > > > remaining
> > > > > > > items are (a) the format of local tier metadata storage and (b)
> > > > > upgrade.
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > > > satish.duggana@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Jun,
> > > > > > > > Thanks for your comments. Please find the inline replies
> below.
> > > > > > > >
> > > > > > > > >605.2 It's rare for the follower to need the remote data.
> So,
> > > the
> > > > > > current
> > > > > > > > approach is fine too. Could you document the process of
> > > rebuilding
> > > > > the
> > > > > > > > producer state since we can't simply trim the producerState
> to an
> > > > > > offset in
> > > > > > > > the middle of a segment.
> > > > > > > >
> > > > > > > > Will clarify in the KIP.
> > > > > > > >
> > > > > > > > >5102.2 Would it be clearer to make startPosiont long and
> > > endPosition
> > > > > > of
> > > > > > > > Optional<Long>?
> > > > > > > >
> > > > > > > > We will have arg checks with respective validation. It is
> not a
> > > good
> > > > > > > > practice to have arguments with optional as mentioned here.
> > > > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > > > >
> > > > > > > >
> > > > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as File
> > > instead of
> > > > > > > > ByteBuffer.
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > > >5102.7 Could you define all public methods for
> LogSegmentData?
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > > >5103.5 Could you change the reference to
> > > rlm_process_interval_ms and
> > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> retry
> > > > > interval
> > > > > > > > config seems still missing. It would be useful to support
> > > exponential
> > > > > > > > backoff with the retry interval config.
> > > > > > > >
> > > > > > > > Good point. We wanted the retry with truncated exponential
> > > backoff,
> > > > > > > > updated the KIP.
> > > > > > > >
> > > > > > > > >5111. "RLM follower fetches the earliest offset for the
> earliest
> > > > > > leader
> > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > topicPartition,
> > > > > > int
> > > > > > > > leaderEpoch) and updates that as the log start offset." This
> > > text is
> > > > > > still
> > > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > > >5115. There are still references to "remote log cleaners".
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > > >6000. Since we are returning new error codes, we need to
> bump
> > > up the
> > > > > > > > protocol version for Fetch request. Also, it will be useful
> to
> > > > > > document all
> > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > >
> > > > > > > > Sure, we will add that in the KIP.
> > > > > > > >
> > > > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs():
> Currently,
> > > > > > leaderEpoch
> > > > > > > > is int32 instead of long.
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed
> > > given
> > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > >
> > > > > > > > No, it is fixed.
> > > > > > > >
> > > > > > > > >6003. RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > topicPartition, long offset, int epochForOffset): Should this
> > > return
> > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > >
> > > > > > > > That makes sense, updated.
> > > > > > > >
> > > > > > > > >6005. RemoteLogState: It seems it's better to split it
> between
> > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> since
> > > the
> > > > > > states
> > > > > > > > are never shared between the two use cases.
> > > > > > > >
> > > > > > > > Agree with that, updated.
> > > > > > > >
> > > > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > > However,
> > > > > > is it
> > > > > > > > ture that other than the metadata topic, RLMM just needs to
> know
> > > > > > whether
> > > > > > > > there is a replica assigned to this broker and doesn't need
> to
> > > know
> > > > > > whether
> > > > > > > > the replica is the leader or the follower?
> > > > > > > >
> > > > > > > > That may be true. If the implementation does not need that,
> it
> > > can
> > > > > > > > ignore the information in the callback.
> > > > > > > >
> > > > > > > > >6007: "Handle expired remote segments (leader and
> follower)":
> > > Why is
> > > > > > this
> > > > > > > > needed in both the leader and the follower?
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > > > >                 "type": "int64",
> > > > > > > > The segment size can just be int32.
> > > > > > > >
> > > > > > > > Updated.
> > > > > > > >
> > > > > > > > >6009. For the record format in the log, it seems that we
> need
> > > to add
> > > > > > > > record
> > > > > > > > type and record version before the serialized bytes. We can
> > > follow
> > > > > the
> > > > > > > > convention used in
> > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > >
> > > > > > > > Yes, KIP already mentions that these are serialized before
> the
> > > > > payload
> > > > > > > > as below. We will mention explicitly that these two are
> written
> > > > > before
> > > > > > > > the data is written.
> > > > > > > >
> > > > > > > > RLMM instance on broker publishes the message to the topic
> with
> > > key
> > > > > as
> > > > > > > > null and value with the below format.
> > > > > > > >
> > > > > > > > type      : unsigned var int, represents the value type. This
> > > value
> > > > > is
> > > > > > > > 'apikey' as mentioned in the schema.
> > > > > > > > version : unsigned var int, the 'version' number of the type
> as
> > > > > > > > mentioned in the schema.
> > > > > > > > data      : record payload in kafka protocol message format.
> > > > > > > >
> > > > > > > >
> > > > > > > > >6010. remote.log.manager.thread.pool.size: The default
> value is
> > > 10.
> > > > > > This
> > > > > > > > might be too high when enabling the tiered feature for the
> first
> > > > > time.
> > > > > > > > Since there are lots of segments that need to be tiered
> > > initially, a
> > > > > > large
> > > > > > > > number of threads could overwhelm the broker.
> > > > > > > >
> > > > > > > > Is the default value 5 reasonable?
> > > > > > > >
> > > > > > > > 6011. "The number of milli seconds to keep the local log
> segment
> > > > > > before it
> > > > > > > > gets deleted. If not set, the value in
> `log.retention.minutes` is
> > > > > > used. If
> > > > > > > > set to -1, no time limit is applied." We should use
> > > log.retention.ms
> > > > > > > > instead of log.retention.minutes.
> > > > > > > > Nice typo catch. Updated the KIP.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Satish.
> > > > > > > >
> > > > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > > > >
> > > > > > > > > Hi, Satish,
> > > > > > > > >
> > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > >
> > > > > > > > > 605.2 It's rare for the follower to need the remote data.
> So,
> > > the
> > > > > > current
> > > > > > > > > approach is fine too. Could you document the process of
> > > rebuilding
> > > > > > the
> > > > > > > > > producer state since we can't simply trim the
> producerState to
> > > an
> > > > > > offset
> > > > > > > > in
> > > > > > > > > the middle of a segment.
> > > > > > > > >
> > > > > > > > > 5102.2 Would it be clearer to make startPosiont long and
> > > > > endPosition
> > > > > > of
> > > > > > > > > Optional<Long>?
> > > > > > > > >
> > > > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File
> > > instead of
> > > > > > > > > ByteBuffer.
> > > > > > > > >
> > > > > > > > > 5102.7 Could you define all public methods for
> LogSegmentData?
> > > > > > > > >
> > > > > > > > > 5103.5 Could you change the reference to
> > > rlm_process_interval_ms
> > > > > and
> > > > > > > > > rlm_retry_interval_ms to the new config names? Also, the
> retry
> > > > > > interval
> > > > > > > > > config seems still missing. It would be useful to support
> > > > > exponential
> > > > > > > > > backoff with the retry interval config.
> > > > > > > > >
> > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > earliest
> > > > > > leader
> > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > topicPartition,
> > > > > > > > int
> > > > > > > > > leaderEpoch) and updates that as the log start offset."
> This
> > > text
> > > > > is
> > > > > > > > still
> > > > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > > > >
> > > > > > > > > 5115. There are still references to "remote log cleaners".
> > > > > > > > >
> > > > > > > > > 6000. Since we are returning new error codes, we need to
> bump
> > > up
> > > > > the
> > > > > > > > > protocol version for Fetch request. Also, it will be
> useful to
> > > > > > document
> > > > > > > > all
> > > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > > >
> > > > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs():
> Currently,
> > > > > > > > leaderEpoch
> > > > > > > > > is int32 instead of long.
> > > > > > > > >
> > > > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion()
> needed
> > > given
> > > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > > >
> > > > > > > > > 6003. RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > topicPartition, long offset, int epochForOffset): Should
> this
> > > > > return
> > > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > > >
> > > > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to
> > > pick a
> > > > > > more
> > > > > > > > > indicative name so that people understand what epoch this
> is.
> > > > > > > > >
> > > > > > > > > 6005. RemoteLogState: It seems it's better to split it
> between
> > > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate
> since
> > > the
> > > > > > states
> > > > > > > > > are never shared between the two use cases.
> > > > > > > > >
> > > > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > > However,
> > > > > > is it
> > > > > > > > > ture that other than the metadata topic, RLMM just needs to
> > > know
> > > > > > whether
> > > > > > > > > there is a replica assigned to this broker and doesn't
> need to
> > > know
> > > > > > > > whether
> > > > > > > > > the replica is the leader or the follower?
> > > > > > > > >
> > > > > > > > > 6007: "Handle expired remote segments (leader and
> follower)":
> > > Why
> > > > > is
> > > > > > this
> > > > > > > > > needed in both the leader and the follower?
> > > > > > > > >
> > > > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > > > >                 "type": "int64",
> > > > > > > > > The segment size can just be int32.
> > > > > > > > >
> > > > > > > > > 6009. For the record format in the log, it seems that we
> need
> > > to
> > > > > add
> > > > > > > > record
> > > > > > > > > type and record version before the serialized bytes. We can
> > > follow
> > > > > > the
> > > > > > > > > convention used in
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > > .
> > > > > > > > >
> > > > > > > > > 6010. remote.log.manager.thread.pool.size: The default
> value
> > > is 10.
> > > > > > This
> > > > > > > > > might be too high when enabling the tiered feature for the
> > > first
> > > > > > time.
> > > > > > > > > Since there are lots of segments that need to be tiered
> > > initially,
> > > > > a
> > > > > > > > large
> > > > > > > > > number of threads could overwhelm the broker.
> > > > > > > > >
> > > > > > > > > 6011. "The number of milli seconds to keep the local log
> > > segment
> > > > > > before
> > > > > > > > it
> > > > > > > > > gets deleted. If not set, the value in
> `log.retention.minutes`
> > > is
> > > > > > used.
> > > > > > > > If
> > > > > > > > > set to -1, no time limit is applied." We should use
> > > > > log.retention.ms
> > > > > > > > > instead of log.retention.minutes.
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > > > satish.duggana@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi,
> > > > > > > > > > We updated the KIP with the points mentioned in the
> earlier
> > > mail
> > > > > > > > > > except for KIP-516 related changes. You can go through
> them
> > > and
> > > > > > let us
> > > > > > > > > > know if you have any comments. We will update the KIP
> with
> > > the
> > > > > > > > > > remaining todo items and KIP-516 related changes by end
> of
> > > this
> > > > > > > > > > week(5th Dec).
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Satish.
> > > > > > > > > >
> > > > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > Hi Jun,
> > > > > > > > > > > Thanks for your comments. Please find the inline
> replies
> > > below.
> > > > > > > > > > >
> > > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting
> the
> > > leader
> > > > > > epoch
> > > > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > > > > mentioned an
> > > > > > > > > > issue
> > > > > > > > > > > earlier. Suppose the leader's local start offset is
> 100.
> > > The
> > > > > > follower
> > > > > > > > > > finds
> > > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > > producerState
> > > > > > > > with
> > > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > > producerState to
> > > > > > > > > > > offset 100 requires more work since one needs to
> download
> > > the
> > > > > > > > previous
> > > > > > > > > > > producerState up to offset 80 and then replay the
> messages
> > > from
> > > > > > 80 to
> > > > > > > > > > 100.
> > > > > > > > > > > It seems that it's simpler in this case for the
> follower
> > > just
> > > > > to
> > > > > > > > take the
> > > > > > > > > > > remote segment as it is and start fetching from offset
> 120.
> > > > > > > > > > >
> > > > > > > > > > > We chose that approach to avoid any edge cases here. It
> > > may be
> > > > > > > > > > > possible that the remote log segment that is received
> may
> > > not
> > > > > > have
> > > > > > > > the
> > > > > > > > > > > same leader epoch sequence from 100-120 as it contains
> on
> > > the
> > > > > > > > > > > leader(this can happen due to unclean leader). It is
> safe
> > > to
> > > > > > start
> > > > > > > > > > > from what the leader returns here.Another way is to
> find
> > > the
> > > > > > remote
> > > > > > > > > > > log segment
> > > > > > > > > > >
> > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems
> that
> > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on
> the
> > > > > > replicas
> > > > > > > > for a
> > > > > > > > > > > partition, not on the replicas for the
> > > > > > __remote_log_segment_metadata
> > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > > > >
> > > > > > > > > > > RLMM will always receive the callback for the remote
> log
> > > > > metadata
> > > > > > > > > > > topic partitions hosted on the local broker and these
> will
> > > be
> > > > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > > > >
> > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> implemented
> > > now.
> > > > > > Could
> > > > > > > > you
> > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > >
> > > > > > > > > > > We mentioned KIP-516 and how it helps. We will update
> this
> > > KIP
> > > > > > with
> > > > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > > > >
> > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > following two
> > > > > > APIs
> > > > > > > > are
> > > > > > > > > > > used. According to the wiki, the former is used for
> topic
> > > > > > deletion
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > latter is used for retention. It seems that retention
> > > should
> > > > > use
> > > > > > the
> > > > > > > > > > former
> > > > > > > > > > > since remote segments without a matching epoch in the
> > > leader
> > > > > > > > (potentially
> > > > > > > > > > > due to unclean leader election) also need to be garbage
> > > > > > collected.
> > > > > > > > The
> > > > > > > > > > > latter seems to be used for the new leader to
> determine the
> > > > > last
> > > > > > > > tiered
> > > > > > > > > > > segment.
> > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > >
> > > > > > > > > > > Right,.that is what we are currently doing. We will
> update
> > > the
> > > > > > > > > > > javadocs and wiki with that. Earlier, we did not want
> to
> > > remove
> > > > > > the
> > > > > > > > > > > segments which are not matched with leader epochs from
> the
> > > > > ladder
> > > > > > > > > > > partition as they may be used later by a replica which
> can
> > > > > > become a
> > > > > > > > > > > leader (unclean leader election) and refer those
> segments.
> > > But
> > > > > > that
> > > > > > > > > > > may leak these segments in remote storage until the
> topic
> > > > > > lifetime.
> > > > > > > > We
> > > > > > > > > > > decided to cleanup the segments with the oldest incase
> of
> > > size
> > > > > > based
> > > > > > > > > > > retention also.
> > > > > > > > > > >
> > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems
> > > that
> > > > > > they can
> > > > > > > > > > > use RemoteLogSegmentId instead of
> RemoteLogSegmentMetadata.
> > > > > > > > > > >
> > > > > > > > > > > It will be useful to have metadata for RSM to fetch log
> > > > > segment.
> > > > > > It
> > > > > > > > > > > may create location/path using id with other metadata
> too.
> > > > > > > > > > >
> > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > > instead of
> > > > > > Long?
> > > > > > > > > > >
> > > > > > > > > > > Wanted to keep endPosition as optional to read till the
> > > end of
> > > > > > the
> > > > > > > > > > > segment and avoid sentinels.
> > > > > > > > > > >
> > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > implementation
> > > > > > and
> > > > > > > > > > others
> > > > > > > > > > > Don't?
> > > > > > > > > > >
> > > > > > > > > > > Actually,  RSM will not have any default
> implementations.
> > > > > Those 3
> > > > > > > > > > > methods were made default earlier for tests etc.
> Updated
> > > the
> > > > > > wiki.
> > > > > > > > > > >
> > > > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > >
> > > > > > > > > > > Sure, they will be added.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> pass
> > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than
> a
> > > file
> > > > > > since
> > > > > > > > it
> > > > > > > > > > will
> > > > > > > > > > > be generated in memory.
> > > > > > > > > > >
> > > > > > > > > > > Right, this is in plan.
> > > > > > > > > > >
> > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs
> > > both
> > > > > > > > baseOffset
> > > > > > > > > > and
> > > > > > > > > > > startOffset. For example, deleteRecords() could move
> the
> > > > > > startOffset
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > middle of a segment. If we copy the full segment to
> remote
> > > > > > storage,
> > > > > > > > the
> > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > >
> > > > > > > > > > > Good point. startOffset is baseOffset by default, if
> not
> > > set
> > > > > > > > explicitly.
> > > > > > > > > > >
> > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > >
> > > > > > > > > > > Sure, updated the wiki.
> > > > > > > > > > >
> > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > >
> > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > >
> > > > > > > > > > > 5103. configs:
> > > > > > > > > > > 5103.1 Could we define the default value of
> non-required
> > > > > configs
> > > > > > > > (e.g the
> > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > >
> > > > > > > > > > > Sure, that makes sense.
> > > > > > > > > > >
> > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > > default to
> > > > > > > > > > retention.ms,
> > > > > > > > > > > instead of remote.log.retention.minutes. Similarly, it
> > > seems
> > > > > > > > > > > that local.log.retention.bytes should default to
> > > segment.bytes.
> > > > > > > > > > >
> > > > > > > > > > > Right, we do not have  remote.log.retention as we
> discussed
> > > > > > earlier.
> > > > > > > > > > > Thanks for catching the typo.
> > > > > > > > > > >
> > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> description
> > > > > says
> > > > > > > > "used in
> > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > > indexes and
> > > > > > > > clean up
> > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > config remote.log.reader.threads for fetching remote
> data.
> > > It's
> > > > > > > > weird to
> > > > > > > > > > > fetch remote index and log in different thread pools
> since
> > > both
> > > > > > are
> > > > > > > > used
> > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > >
> > > > > > > > > > > Right, remote.log.manager.thread.pool is mainly used
> for
> > > > > > copy/cleanup
> > > > > > > > > > > activities. Fetch path always goes through
> > > > > > remote.log.reader.threads.
> > > > > > > > > > >
> > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that
> the
> > > amount
> > > > > > of
> > > > > > > > time
> > > > > > > > > > to
> > > > > > > > > > > back off when there is no work to do? If so, perhaps it
> > > can be
> > > > > > > > renamed as
> > > > > > > > > > > backoff.ms.
> > > > > > > > > > >
> > > > > > > > > > > This is the delay interval for each iteration. It may
> be
> > > > > renamed
> > > > > > to
> > > > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > > > >
> > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > rlm_retry_interval_ms
> > > > > > > > configs? If
> > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > >
> > > > > > > > > > > remote.log.manager.task.interval.ms is the process
> > > internal,
> > > > > > retry
> > > > > > > > > > > interval is missing in the configs, which will be
> updated
> > > in
> > > > > the
> > > > > > KIP.
> > > > > > > > > > >
> > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of
> the
> > > index
> > > > > > > > files of
> > > > > > > > > > > remote log segments to avoid multiple index fetches
> from
> > > the
> > > > > > remote
> > > > > > > > > > > storage." Is the RLM in memory or on disk? If on disk,
> > > where is
> > > > > > it
> > > > > > > > > > stored?
> > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > >
> > > > > > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > > > > > `remote-log-index-cache` under log dir. We plan to
> have a
> > > > > config
> > > > > > for
> > > > > > > > > > > that instead of default. We will have a configuration
> for
> > > that.
> > > > > > > > > > >
> > > > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest
> > > Local
> > > > > > Offset
> > > > > > > > in
> > > > > > > > > > > different places. It would be useful to standardize the
> > > > > > terminology.
> > > > > > > > > > >
> > > > > > > > > > > Sure.
> > > > > > > > > > >
> > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state".
> It
> > > listed
> > > > > > two
> > > > > > > > > > options
> > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > > > > > > >
> > > > > > > > > > > 5107. Follower to leader transition: It has step 2,
> but not
> > > > > step
> > > > > > 1.
> > > > > > > > > > > Step-1 is there but it is not explicitly highlighted.
> It is
> > > > > > previous
> > > > > > > > > > > table to step-2.
> > > > > > > > > > >
> > > > > > > > > > > 5108. If a consumer fetches from the remote data and
> the
> > > remote
> > > > > > > > storage
> > > > > > > > > > is
> > > > > > > > > > > not available, what error code is used in the fetch
> > > response?
> > > > > > > > > > >
> > > > > > > > > > > Good point. We have not yet defined the error for this
> > > case. We
> > > > > > need
> > > > > > > > > > > to define an error message and send the same in fetch
> > > response.
> > > > > > > > > > >
> > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the
> > > first
> > > > > > message
> > > > > > > > > > > offset whose timestamp is >= to the given timestamp in
> the
> > > > > > request.
> > > > > > > > That
> > > > > > > > > > > means it checks in remote log time indexes first, after
> > > which
> > > > > > local
> > > > > > > > log
> > > > > > > > > > > time indexes are checked." Could you document which
> method
> > > in
> > > > > > RLMM is
> > > > > > > > > > used
> > > > > > > > > > > for this?
> > > > > > > > > > >
> > > > > > > > > > > Okay.
> > > > > > > > > > >
> > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > > metadata
> > > > > > of
> > > > > > > > that
> > > > > > > > > > > partition with a delete marker and publishes them to
> RLMM."
> > > > > This
> > > > > > > > seems
> > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > >
> > > > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > > > >
> > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > > earliest
> > > > > > > > leader
> > > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > > topicPartition,
> > > > > > > > > > int
> > > > > > > > > > > leaderEpoch) and updates that as the log start
> offset." Do
> > > we
> > > > > > need
> > > > > > > > that
> > > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > > >
> > > > > > > > > > > Good point. Right, existing replication protocol takes
> > > care of
> > > > > > > > > > > updating the followers’s log start offset received
> from the
> > > > > > leader.
> > > > > > > > > > >
> > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> fetching
> > > > > from
> > > > > > > > remote
> > > > > > > > > > > storage?
> > > > > > > > > > >
> > > > > > > > > > > Remote reads may fail within the current default wait
> > > time, but
> > > > > > > > > > > subsequent fetches would be able to serve as that data
> is
> > > > > stored
> > > > > > in
> > > > > > > > > > > the local cache. This cache is currently implemented in
> > > RSMs.
> > > > > > But we
> > > > > > > > > > > plan to pull this into the remote log messaging layer
> in
> > > > > future.
> > > > > > > > > > >
> > > > > > > > > > > 5113. "Committed offsets can be stored in a local file
> to
> > > avoid
> > > > > > > > reading
> > > > > > > > > > the
> > > > > > > > > > > messages again when a broker is restarted." Could you
> > > describe
> > > > > > the
> > > > > > > > format
> > > > > > > > > > > and the location of the file? Also, could the same
> message
> > > be
> > > > > > > > processed
> > > > > > > > > > by
> > > > > > > > > > > RLMM again after broker restart? If so, how do we
> handle
> > > that?
> > > > > > > > > > >
> > > > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > > > >
> > > > > > > > > > > 5114. Message format
> > > > > > > > > > > 5114.1 There are two records named
> > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > with
> > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > >
> > > > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > > > >
> > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> document
> > > > > whether
> > > > > > > > > > endOffset
> > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > It is inclusive, will update.
> > > > > > > > > > >
> > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> explain
> > > > > > LeaderEpoch
> > > > > > > > a
> > > > > > > > > > bit
> > > > > > > > > > > more? Is that the epoch of the leader when it copies
> the
> > > > > segment
> > > > > > to
> > > > > > > > > > remote
> > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > >
> > > > > > > > > > > Right, this is the leader epoch of the broker which
> copied
> > > this
> > > > > > > > > > > segment. This is helpful in reason about which broker
> > > copied
> > > > > the
> > > > > > > > > > > segment to remote storage.
> > > > > > > > > > >
> > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> more?
> > > Each
> > > > > > > > record in
> > > > > > > > > > > Kafka already has a timestamp field. Could we just use
> > > that?
> > > > > > > > > > >
> > > > > > > > > > > This is the  timestamp at which  the respective event
> > > occurred.
> > > > > > Added
> > > > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any
> other
> > > > > > > > > > > implementation. We thought about that but it looked
> > > cleaner to
> > > > > > use at
> > > > > > > > > > > the message structure level instead of getting that
> from
> > > the
> > > > > > consumer
> > > > > > > > > > > record and using that to build the respective event.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > > >
> > > > > > > > > > > Right, it looks like config allows only int value >=
> 14.
> > > > > > > > > > >
> > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> with
> > > the
> > > > > log
> > > > > > > > cleaner
> > > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > >
> > > > > > > > > > > I am fine with RemotePartitionRemover or
> > > > > > RemoteLogDeletionManager(we
> > > > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > > > >
> > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > > processes
> > > > > it
> > > > > > if
> > > > > > > > it is
> > > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > >
> > > > > > > > > > > This is to handle duplicate delete_partition_marked
> > > events. RLC
> > > > > > > > > > > internally maintains a state for the delete_partition
> > > events
> > > > > and
> > > > > > if
> > > > > > > > it
> > > > > > > > > > > already has an existing event then it ignores if it is
> > > already
> > > > > > being
> > > > > > > > > > > processed.
> > > > > > > > > > >
> > > > > > > > > > > 5117. Should we add a new MessageFormatter to read the
> tier
> > > > > > metadata
> > > > > > > > > > topic?
> > > > > > > > > > >
> > > > > > > > > > > Right, this is in plan but did not mention it in the
> KIP.
> > > This
> > > > > > will
> > > > > > > > be
> > > > > > > > > > > useful for debugging purposes too.
> > > > > > > > > > >
> > > > > > > > > > > 5118. "Maximum remote log reader thread pool task queue
> > > size.
> > > > > If
> > > > > > the
> > > > > > > > task
> > > > > > > > > > > queue is full, broker will stop reading remote log
> > > segments."
> > > > > > What
> > > > > > > > do we
> > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > >
> > > > > > > > > > > We return an error response for that partition.
> > > > > > > > > > >
> > > > > > > > > > > 5119. It would be useful to list all things not
> supported
> > > in
> > > > > the
> > > > > > > > first
> > > > > > > > > > > version in a Future work or Limitations section. For
> > > example,
> > > > > > > > compacted
> > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> true
> > > to
> > > > > > false,
> > > > > > > > etc.
> > > > > > > > > > >
> > > > > > > > > > > We already have a non-goals section which is filled
> with
> > > some
> > > > > of
> > > > > > > > these
> > > > > > > > > > > details. Do we need another limitations section?
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Satish.
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <
> jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> below.
> > > > > > > > > > > >
> > > > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting
> the
> > > > > leader
> > > > > > > > epoch
> > > > > > > > > > > > sequence received from remote storage to [LSO,
> ELO]." I
> > > > > > mentioned
> > > > > > > > an
> > > > > > > > > > issue
> > > > > > > > > > > > earlier. Suppose the leader's local start offset is
> 100.
> > > The
> > > > > > > > follower
> > > > > > > > > > finds
> > > > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > > producerState
> > > > > > > > > > with
> > > > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > > producerState
> > > > > > > > to
> > > > > > > > > > > > offset 100 requires more work since one needs to
> > > download the
> > > > > > > > previous
> > > > > > > > > > > > producerState up to offset 80 and then replay the
> > > messages
> > > > > > from 80
> > > > > > > > to
> > > > > > > > > > 100.
> > > > > > > > > > > > It seems that it's simpler in this case for the
> follower
> > > just
> > > > > > to
> > > > > > > > take
> > > > > > > > > > the
> > > > > > > > > > > > remote segment as it is and start fetching from
> offset
> > > 120.
> > > > > > > > > > > >
> > > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems
> that
> > > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called
> on the
> > > > > > replicas
> > > > > > > > for
> > > > > > > > > > a
> > > > > > > > > > > > partition, not on the replicas for the
> > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > > obtains the metadata for remote segments for
> deletion.
> > > > > > > > > > > >
> > > > > > > > > > > > 5100. KIP-516 has been accepted and is being
> implemented
> > > now.
> > > > > > > > Could you
> > > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > > >
> > > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > > following
> > > > > two
> > > > > > > > APIs
> > > > > > > > > > are
> > > > > > > > > > > > used. According to the wiki, the former is used for
> topic
> > > > > > deletion
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > > latter is used for retention. It seems that retention
> > > should
> > > > > > use
> > > > > > > > the
> > > > > > > > > > former
> > > > > > > > > > > > since remote segments without a matching epoch in the
> > > leader
> > > > > > > > > > (potentially
> > > > > > > > > > > > due to unclean leader election) also need to be
> garbage
> > > > > > collected.
> > > > > > > > The
> > > > > > > > > > > > latter seems to be used for the new leader to
> determine
> > > the
> > > > > > last
> > > > > > > > tiered
> > > > > > > > > > > > segment.
> > > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > > >
> > > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it
> seems
> > > that
> > > > > > they
> > > > > > > > can
> > > > > > > > > > > > use RemoteLogSegmentId instead of
> > > RemoteLogSegmentMetadata.
> > > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > > instead
> > > > > of
> > > > > > > > Long?
> > > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > > implementation and
> > > > > > > > > > others
> > > > > > > > > > > > don't?
> > > > > > > > > > > > 5102.4. Could we define
> RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to
> pass
> > > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array
> than a
> > > file
> > > > > > > > since it
> > > > > > > > > > will
> > > > > > > > > > > > be generated in memory.
> > > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it
> needs
> > > both
> > > > > > > > > > baseOffset and
> > > > > > > > > > > > startOffset. For example, deleteRecords() could move
> the
> > > > > > > > startOffset
> > > > > > > > > > to the
> > > > > > > > > > > > middle of a segment. If we copy the full segment to
> > > remote
> > > > > > > > storage, the
> > > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > is
> > > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > > >
> > > > > > > > > > > > 5103. configs:
> > > > > > > > > > > > 5103.1 Could we define the default value of
> non-required
> > > > > > configs
> > > > > > > > (e.g
> > > > > > > > > > the
> > > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > > default
> > > > > to
> > > > > > > > > > retention.ms,
> > > > > > > > > > > > instead of remote.log.retention.minutes. Similarly,
> it
> > > seems
> > > > > > > > > > > > that local.log.retention.bytes should default to
> > > > > segment.bytes.
> > > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > > description
> > > > > > says
> > > > > > > > "used
> > > > > > > > > > in
> > > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > > indexes
> > > > > and
> > > > > > > > clean
> > > > > > > > > > up
> > > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > > config remote.log.reader.threads for fetching remote
> > > data.
> > > > > It's
> > > > > > > > weird
> > > > > > > > > > to
> > > > > > > > > > > > fetch remote index and log in different thread pools
> > > since
> > > > > > both are
> > > > > > > > > > used
> > > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that
> the
> > > > > > amount of
> > > > > > > > > > time to
> > > > > > > > > > > > back off when there is no work to do? If so, perhaps
> it
> > > can
> > > > > be
> > > > > > > > renamed
> > > > > > > > > > as
> > > > > > > > > > > > backoff.ms.
> > > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > > rlm_retry_interval_ms
> > > > > > > > configs?
> > > > > > > > > > If
> > > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > > >
> > > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU)
> of the
> > > > > index
> > > > > > > > files
> > > > > > > > > > of
> > > > > > > > > > > > remote log segments to avoid multiple index fetches
> from
> > > the
> > > > > > remote
> > > > > > > > > > > > storage." Is the RLM in memory or on disk? If on
> disk,
> > > where
> > > > > > is it
> > > > > > > > > > stored?
> > > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > > >
> > > > > > > > > > > > 5105. The KIP uses local-log-start-offset and
> Earliest
> > > Local
> > > > > > > > Offset in
> > > > > > > > > > > > different places. It would be useful to standardize
> the
> > > > > > > > terminology.
> > > > > > > > > > > >
> > > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux
> state". It
> > > > > > listed two
> > > > > > > > > > options
> > > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > > >
> > > > > > > > > > > > 5107. Follower to leader transition: It has step 2,
> but
> > > not
> > > > > > step 1.
> > > > > > > > > > > >
> > > > > > > > > > > > 5108. If a consumer fetches from the remote data and
> the
> > > > > remote
> > > > > > > > > > storage is
> > > > > > > > > > > > not available, what error code is used in the fetch
> > > response?
> > > > > > > > > > > >
> > > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns
> the
> > > first
> > > > > > > > message
> > > > > > > > > > > > offset whose timestamp is >= to the given timestamp
> in
> > > the
> > > > > > request.
> > > > > > > > > > That
> > > > > > > > > > > > means it checks in remote log time indexes first,
> after
> > > which
> > > > > > > > local log
> > > > > > > > > > > > time indexes are checked." Could you document which
> > > method in
> > > > > > RLMM
> > > > > > > > is
> > > > > > > > > > used
> > > > > > > > > > > > for this?
> > > > > > > > > > > >
> > > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log
> segment
> > > > > > metadata of
> > > > > > > > that
> > > > > > > > > > > > partition with a delete marker and publishes them to
> > > RLMM."
> > > > > > This
> > > > > > > > seems
> > > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > > >
> > > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for
> the
> > > > > > earliest
> > > > > > > > leader
> > > > > > > > > > > > epoch by calling
> RLMM.earliestLogOffset(TopicPartition
> > > > > > > > topicPartition,
> > > > > > > > > > int
> > > > > > > > > > > > leaderEpoch) and updates that as the log start
> offset."
> > > Do we
> > > > > > need
> > > > > > > > that
> > > > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > > > >
> > > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > > fetching
> > > > > > from
> > > > > > > > remote
> > > > > > > > > > > > storage?
> > > > > > > > > > > >
> > > > > > > > > > > > 5113. "Committed offsets can be stored in a local
> file to
> > > > > avoid
> > > > > > > > > > reading the
> > > > > > > > > > > > messages again when a broker is restarted." Could you
> > > > > describe
> > > > > > the
> > > > > > > > > > format
> > > > > > > > > > > > and the location of the file? Also, could the same
> > > message be
> > > > > > > > > > processed by
> > > > > > > > > > > > RLMM again after broker restart? If so, how do we
> handle
> > > > > that?
> > > > > > > > > > > >
> > > > > > > > > > > > 5114. Message format
> > > > > > > > > > > > 5114.1 There are two records named
> > > > > > RemoteLogSegmentMetadataRecord
> > > > > > > > with
> > > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we
> document
> > > > > > whether
> > > > > > > > > > endOffset
> > > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you
> explain
> > > > > > > > LeaderEpoch a
> > > > > > > > > > bit
> > > > > > > > > > > > more? Is that the epoch of the leader when it copies
> the
> > > > > > segment to
> > > > > > > > > > remote
> > > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit
> more?
> > > > > Each
> > > > > > > > record
> > > > > > > > > > in
> > > > > > > > > > > > Kafka already has a timestamp field. Could we just
> use
> > > that?
> > > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > > > >
> > > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused
> with
> > > the
> > > > > > log
> > > > > > > > > > cleaner
> > > > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > > >
> > > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > > processes
> > > > > > it
> > > > > > > > if it
> > > > > > > > > > is
> > > > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > > >
> > > > > > > > > > > > 5117. Should we add a new MessageFormatter to read
> the
> > > tier
> > > > > > > > metadata
> > > > > > > > > > topic?
> > > > > > > > > > > >
> > > > > > > > > > > > 5118. "Maximum remote log reader thread pool task
> queue
> > > size.
> > > > > > If
> > > > > > > > the
> > > > > > > > > > task
> > > > > > > > > > > > queue is full, broker will stop reading remote log
> > > segments."
> > > > > > What
> > > > > > > > do
> > > > > > > > > > we
> > > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > > >
> > > > > > > > > > > > 5119. It would be useful to list all things not
> > > supported in
> > > > > > the
> > > > > > > > first
> > > > > > > > > > > > version in a Future work or Limitations section. For
> > > example,
> > > > > > > > compacted
> > > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> > > true to
> > > > > > false,
> > > > > > > > > > etc.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > > > (
> > > > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > > > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Jun,

Thanks for your comments. Please find the inline replies below.
Several of these were discussed/clarified in our last discussion.

6000. Since we are returning new error codes, we need to bump up the
protocol version for Fetch request. Also, it will be useful to document all
new error codes and whether they are retriable or not.
This previous comment doesn't seem to be addressed.

Updated KIP on fetch protocol and error code.

9110. flat_file_format: Could you define the serialized representation for
each field in the header?

Added the serialized representation types in the KIP.

9111. RPM has the following 2 steps.
"1. The controller publishes delete_partition_marked event to say that the
partition is marked for deletion. There can be multiple events published
when the controller restarts or failover and this event will be
deduplicated by RPM.
2. RPM receives the delete_partition_marked and processes it if it is not
yet processed earlier."
What triggers RPM to read __remote_log_metadata? Is RPM part of the default
RLMM implementation or is it meant for any RLMM implementation?

RPM(or RPRM) is only for the default RLMM implementation. RPM receives
the delete_partition_marked events from RLMM and acts on them. Updated
KIP with more details.

9112. remote.log.manager.task.retry.backoff.ms: It would be useful to make
it clear in the comment that this is for the initial retry backoff.

Updated the KIP.

9113. RLMM:
9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be updated.

Updated javadoc of this method.

9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
since all other update events have leaderEpoch?

+1 to have that,  updated the KIP.

9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
to make it clear this is for deletion?

Sure, updated the KIP.

9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
clear this is for segment?

Sure, Updated the KIP.

9114.Upgrade:
9114.1 It seems that we require message format > 0.11 for turning on the
remote store feature.

9114.2 It's not clear to me why remote.log.storage.system.enable needs to
be set to true before bumping up inter.broker.protocol.version.

Agree that based on our discussion, this is not required. Upgrade
notes is updated.

9114.3 "If the topic-id is not received in the LeaderAndIsr request then
remote log storage will not start. But it will log an error message in the
log. One way to address this is to do a rolling restart of that broker, so
that the leader will be moved to another broker and the controller will
send LeaderAndIsr with the registered topic-id." Why is this needed? With
KIP-516, after upgrading to the latest protocol, topicIds are auto created.

I thought there may be edge cases of not receiving topic-id but you
clarified in the meeting that won’t be the case. I agree that it is
not needed.

9115. testing: Could you comment on how we plan to do integration and
system tests? Do we plan to include, for example, an in-memory
implementation of RSM?

For integration tests, we can have file based RSM that we have. For
system tests, we can have a single node HDFS cluster in one of the
containers and use HDFS RSM implementation.

9116. There is still a todo for the message formatter.

Updated the KIP with the format.


Satish.


On Tue, 12 Jan 2021 at 07:07, Jun Rao <ju...@confluent.io> wrote:
>
> Hi, Satish,
>
> Thanks for the reply. A few more followup comments.
>
> 6000. Since we are returning new error codes, we need to bump up the
> protocol version for Fetch request. Also, it will be useful to document all
> new error codes and whether they are retriable or not.
> This previous comment doesn't seem to be addressed.
>
> 9110. flat_file_format: Could you define the serialized representation for
> each field in the header?
>
> 9111. RPM has the following 2 steps.
> "1. The controller publishes delete_partition_marked event to say that the
> partition is marked for deletion. There can be multiple events published
> when the controller restarts or failover and this event will be
> deduplicated by RPM.
> 2. RPM receives the delete_partition_marked and processes it if it is not
> yet processed earlier."
> What triggers RPM to read __remote_log_metadata? Is RPM part of the default
> RLMM implementation or is it meant for any RLMM implementation?
>
> 9112. remote.log.manager.task.retry.backoff.ms: It would be useful to make
> it clear in the comment that this is for the initial retry backoff.
>
> 9113. RLMM:
> 9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be updated.
> 9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
> since all other update events have leaderEpoch?
> 9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
> to make it clear this is for deletion?
> 9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
> clear this is for segment?
>
> 9114.Upgrade:
> 9114.1 It seems that we require message format > 0.11 for turning on the
> remote store feature.
> 9114.2 It's not clear to me why remote.log.storage.system.enable needs to
> be set to true before bumping up inter.broker.protocol.version.
> 9114.3 "If the topic-id is not received in the LeaderAndIsr request then
> remote log storage will not start. But it will log an error message in the
> log. One way to address this is to do a rolling restart of that broker, so
> that the leader will be moved to another broker and the controller will
> send LeaderAndIsr with the registered topic-id." Why is this needed? With
> KIP-516, after upgrading to the latest protocol, topicIds are auto created.
>
> 9115. testing: Could you comment on how we plan to do integration and
> system tests? Do we plan to include, for example, an in-memory
> implementation of RSM?
>
> 9116. There is still a todo for the message formatter.
>
> Jun
>
> On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi Jun,
> > Thanks for your comments. Please find the inline replies below.
> >
> > 6022. For packages used for server plugins, the convention is to
> > use org.apache.kafka.server. See java-based Authorizer as an example.
> >
> > Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> > ‘org.apache.kafka.server.log.remote.storage’.  Updated in the KIP.
> >
> > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat
> > file captures the state of the remote segments. After a partition is
> > deleted, it seems that we just need to remove the partitions's remote
> > segments from the flat file.
> >
> > DeletePartitionState might not yet have been processed by RPM and not
> > completed. We  will not have  that in flat  file format  once it
> > reaches DELETE_PARTITION_FINISHED state.
> >
> > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > version. It seems that's doable. One can just do the normal upgrade first
> > and wait enough time (for producer snapshots to be built), and then enable
> > remote storage.
> >
> > Upgrade notes updates in the KIP.
> >
> > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > tracking the remote segments when RLMM.onPartitionLeadershipChanges() is
> > called with the broker being the leader for __remote_log_metadata
> > partition? If so, could we document it?
> >
> > The current plan is to have that as part of RLMM and RPM uses that to
> > get the remote segments list. I will add this detail in the respective
> > sections.
> >
> > Satish.
> >
> > On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Hi, Satish,
> > >
> > > Thanks for the reply. A few more followup comments.
> > >
> > > 6022. For packages used for server plugins, the convention is to
> > > use org.apache.kafka.server. See java-based Authorizer as an example.
> > >
> > > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat
> > > file captures the state of the remote segments. After a partition is
> > > deleted, it seems that we just need to remove the partitions's remote
> > > segments from the flat file.
> > >
> > > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > > version. It seems that's doable. One can just do the normal upgrade first
> > > and wait enough time (for producer snapshots to be built), and then
> > enable
> > > remote storage.
> > >
> > > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > > tracking the remote segments when RLMM.onPartitionLeadershipChanges() is
> > > called with the broker being the leader for __remote_log_metadata
> > > partition? If so, could we document it?
> > >
> > > Jun
> > >
> > > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <kprakasam@confluent.io
> > >
> > > wrote:
> > >
> > > > Hi Satish,
> > > >
> > > > Thanks for the updates! A few more comments below.
> > > >
> > > > 9001. Under the "Upgrade" section, there is a line mentioning:
> > "Upgrade the
> > > > existing Kafka cluster to 2.7 version and allow this to run for the log
> > > > retention of user topics that you want to enable tiered storage. This
> > will
> > > > allow all the topics to have the producer snapshots generated for each
> > log
> > > > segment." -- Which associated change in AK were you referring to here?
> > Is
> > > > it: https://github.com/apache/kafka/pull/7929 ? It seems like I don't
> > see
> > > > it in the 2.7 release branch yet, here is the link:
> > > > https://github.com/apache/kafka/commits/2.7.
> > > >
> > > > 9002. Under the "Upgrade" section, the configuration mentioned is
> > > > 'remote.log.storage.system.enable'. However, under "Public Interfaces"
> > > > section the corresponding configuration is
> > 'remote.storage.system.enable'.
> > > > Could we use the same one in both, maybe
> > > > 'remote.log.storage.system.enable'?
> > > >
> > > > 9003. Under "Per Topic Configuration", the KIP recommends setting
> > > > 'remote.log.storage.enable' to true at a per-topic level. It will be
> > useful
> > > > to add a line that if the user wants to enable it for all topics, then
> > they
> > > > should be able to set the cluster-wide default to true. Also, it will
> > be
> > > > useful to mention that the KIP currently does not support setting it to
> > > > false (after it is set to true), and add that to the future work
> > section.
> > > >
> > > > 9004. Under "Committed offsets file format", the sample provided shows
> > > > partition number and offset. Is the topic name required for identifying
> > > > which topic the partitions belong to?
> > > >
> > > > 9005. Under "Internal flat-file store format of remote log metadata",
> > it
> > > > seems useful to specify both topic name and topic ID for debugging
> > > > purposes.
> > > >
> > > > 9006. Under "Internal flat-file store format of remote log metadata",
> > the
> > > > description of "metadata-topic-offset" currently says "offset of the
> > remote
> > > > log metadata topic from which this topic partition's remote log
> > metadata is
> > > > fetched." Just for the wording, perhaps you meant to refer to the
> > offset
> > > > upto which the file has been committed? i.e. "offset of the remote log
> > > > metadata topic upto which this topic partition's remote log metadata
> > has
> > > > been committed into this file."
> > > >
> > > > 9007. Under "Internal flat-file store format of remote log metadata",
> > the
> > > > schema of the payload (i.e. beyond the header) seems to contain the
> > events
> > > > from the metadata topic. It seems useful to instead persist the
> > > > representation of the materialized state of the events, so that for the
> > > > same segment only the latest state is stored. Besides reducing storage
> > > > footprint, this also is likely to relate directly with the in-memory
> > > > representation of the RLMM cache (which probably is some kind of a Map
> > with
> > > > key being segment ID and value being the segment state), so recovery
> > from
> > > > disk will be straightforward.
> > > >
> > > > 9008. Under "Topic deletion lifecycle", step (1), it will be useful to
> > > > mention when in the deletion flow does the controller publish the
> > > > delete_partition_marked event to say that the partition is marked for
> > > > deletion?
> > > >
> > > > 9009. There are ~4 TODOs in the KIP. Could you please address these or
> > > > remove them?
> > > >
> > > > 9010. There is a reference to a Google doc on the KIP which was used
> > > > earlier for discussions. Please could you remove the reference, since
> > the
> > > > KIP is the source of the truth?
> > > >
> > > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > > >
> > > >
> > > > Cheers,
> > > > Kowshik
> > > >
> > > >
> > > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> > satish.duggana@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Jun,
> > > > > Thanks for your comments. Please go through the inline replies.
> > > > >
> > > > >
> > > > > 5102.2: It seems that both positions can just be int. Another option
> > is
> > > > to
> > > > > have two methods. Would it be clearer?
> > > > >
> > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata,  int startPosition)
> > > > throwsRemoteStorageException;
> > > > >
> > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata, int startPosition, int endPosition) throws
> > > > > RemoteStorageException;
> > > > >
> > > > > That makes sense to me, updated the KIP.
> > > > >
> > > > > 6003: Could you also update the javadoc for the return value?
> > > > >
> > > > > Updated.
> > > > >
> > > > > 6020: local.log.retention.bytes: Should it default to
> > log.retention.bytes
> > > > > to be consistent with local.log.retention.ms?
> > > > >
> > > > > Yes, it can be defaulted to log.retention.bytes.
> > > > >
> > > > > 6021: Could you define TopicIdPartition?
> > > > >
> > > > > Added TopicIdPartition in the KIP.
> > > > >
> > > > > 6022: For all public facing classes, could you specify the package
> > name?
> > > > >
> > > > > Updated.
> > > > >
> > > > >
> > > > > Thanks,
> > > > > Satish.
> > > > >
> > > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > Hi, Satish,
> > > > > >
> > > > > > Thanks for the reply. A few more comments below.
> > > > > >
> > > > > > 5102.2: It seems that both positions can just be int. Another
> > option is
> > > > > to
> > > > > > have two methods. Would it be clearer?
> > > > > >
> > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata,
> > > > > >                                     int startPosition) throws
> > > > > > RemoteStorageException;
> > > > > >
> > > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > > remoteLogSegmentMetadata,
> > > > > >                                     int startPosition, int
> > endPosition)
> > > > > > throws RemoteStorageException;
> > > > > >
> > > > > > 6003: Could you also update the javadoc for the return value?
> > > > > >
> > > > > > 6010: What kind of tiering throughput have you seen with 5 threads?
> > > > > >
> > > > > > 6020: local.log.retention.bytes: Should it default to
> > > > log.retention.bytes
> > > > > > to be consistent with local.log.retention.ms?
> > > > > >
> > > > > > 6021: Could you define TopicIdPartition?
> > > > > >
> > > > > > 6022: For all public facing classes, could you specify the package
> > > > name?
> > > > > >
> > > > > > It seems that you already added the topicId support. Two other
> > > > remaining
> > > > > > items are (a) the format of local tier metadata storage and (b)
> > > > upgrade.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Jun,
> > > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > > >
> > > > > > > >605.2 It's rare for the follower to need the remote data. So,
> > the
> > > > > current
> > > > > > > approach is fine too. Could you document the process of
> > rebuilding
> > > > the
> > > > > > > producer state since we can't simply trim the producerState to an
> > > > > offset in
> > > > > > > the middle of a segment.
> > > > > > >
> > > > > > > Will clarify in the KIP.
> > > > > > >
> > > > > > > >5102.2 Would it be clearer to make startPosiont long and
> > endPosition
> > > > > of
> > > > > > > Optional<Long>?
> > > > > > >
> > > > > > > We will have arg checks with respective validation. It is not a
> > good
> > > > > > > practice to have arguments with optional as mentioned here.
> > > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > > >
> > > > > > >
> > > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as File
> > instead of
> > > > > > > ByteBuffer.
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >5102.7 Could you define all public methods for LogSegmentData?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >5103.5 Could you change the reference to
> > rlm_process_interval_ms and
> > > > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > > > interval
> > > > > > > config seems still missing. It would be useful to support
> > exponential
> > > > > > > backoff with the retry interval config.
> > > > > > >
> > > > > > > Good point. We wanted the retry with truncated exponential
> > backoff,
> > > > > > > updated the KIP.
> > > > > > >
> > > > > > > >5111. "RLM follower fetches the earliest offset for the earliest
> > > > > leader
> > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > topicPartition,
> > > > > int
> > > > > > > leaderEpoch) and updates that as the log start offset." This
> > text is
> > > > > still
> > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >5115. There are still references to "remote log cleaners".
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >6000. Since we are returning new error codes, we need to bump
> > up the
> > > > > > > protocol version for Fetch request. Also, it will be useful to
> > > > > document all
> > > > > > > new error codes and whether they are retriable or not.
> > > > > > >
> > > > > > > Sure, we will add that in the KIP.
> > > > > > >
> > > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > > > leaderEpoch
> > > > > > > is int32 instead of long.
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed
> > given
> > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > >
> > > > > > > No, it is fixed.
> > > > > > >
> > > > > > > >6003. RemoteLogSegmentMetadata
> > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > topicPartition, long offset, int epochForOffset): Should this
> > return
> > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > >
> > > > > > > That makes sense, updated.
> > > > > > >
> > > > > > > >6005. RemoteLogState: It seems it's better to split it between
> > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since
> > the
> > > > > states
> > > > > > > are never shared between the two use cases.
> > > > > > >
> > > > > > > Agree with that, updated.
> > > > > > >
> > > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > However,
> > > > > is it
> > > > > > > ture that other than the metadata topic, RLMM just needs to know
> > > > > whether
> > > > > > > there is a replica assigned to this broker and doesn't need to
> > know
> > > > > whether
> > > > > > > the replica is the leader or the follower?
> > > > > > >
> > > > > > > That may be true. If the implementation does not need that, it
> > can
> > > > > > > ignore the information in the callback.
> > > > > > >
> > > > > > > >6007: "Handle expired remote segments (leader and follower)":
> > Why is
> > > > > this
> > > > > > > needed in both the leader and the follower?
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > > >                 "type": "int64",
> > > > > > > The segment size can just be int32.
> > > > > > >
> > > > > > > Updated.
> > > > > > >
> > > > > > > >6009. For the record format in the log, it seems that we need
> > to add
> > > > > > > record
> > > > > > > type and record version before the serialized bytes. We can
> > follow
> > > > the
> > > > > > > convention used in
> > > > > > >
> > > > > > >
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > >
> > > > > > > Yes, KIP already mentions that these are serialized before the
> > > > payload
> > > > > > > as below. We will mention explicitly that these two are written
> > > > before
> > > > > > > the data is written.
> > > > > > >
> > > > > > > RLMM instance on broker publishes the message to the topic with
> > key
> > > > as
> > > > > > > null and value with the below format.
> > > > > > >
> > > > > > > type      : unsigned var int, represents the value type. This
> > value
> > > > is
> > > > > > > 'apikey' as mentioned in the schema.
> > > > > > > version : unsigned var int, the 'version' number of the type as
> > > > > > > mentioned in the schema.
> > > > > > > data      : record payload in kafka protocol message format.
> > > > > > >
> > > > > > >
> > > > > > > >6010. remote.log.manager.thread.pool.size: The default value is
> > 10.
> > > > > This
> > > > > > > might be too high when enabling the tiered feature for the first
> > > > time.
> > > > > > > Since there are lots of segments that need to be tiered
> > initially, a
> > > > > large
> > > > > > > number of threads could overwhelm the broker.
> > > > > > >
> > > > > > > Is the default value 5 reasonable?
> > > > > > >
> > > > > > > 6011. "The number of milli seconds to keep the local log segment
> > > > > before it
> > > > > > > gets deleted. If not set, the value in `log.retention.minutes` is
> > > > > used. If
> > > > > > > set to -1, no time limit is applied." We should use
> > log.retention.ms
> > > > > > > instead of log.retention.minutes.
> > > > > > > Nice typo catch. Updated the KIP.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > >
> > > > > > > > 605.2 It's rare for the follower to need the remote data. So,
> > the
> > > > > current
> > > > > > > > approach is fine too. Could you document the process of
> > rebuilding
> > > > > the
> > > > > > > > producer state since we can't simply trim the producerState to
> > an
> > > > > offset
> > > > > > > in
> > > > > > > > the middle of a segment.
> > > > > > > >
> > > > > > > > 5102.2 Would it be clearer to make startPosiont long and
> > > > endPosition
> > > > > of
> > > > > > > > Optional<Long>?
> > > > > > > >
> > > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File
> > instead of
> > > > > > > > ByteBuffer.
> > > > > > > >
> > > > > > > > 5102.7 Could you define all public methods for LogSegmentData?
> > > > > > > >
> > > > > > > > 5103.5 Could you change the reference to
> > rlm_process_interval_ms
> > > > and
> > > > > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > > > > interval
> > > > > > > > config seems still missing. It would be useful to support
> > > > exponential
> > > > > > > > backoff with the retry interval config.
> > > > > > > >
> > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > earliest
> > > > > leader
> > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > topicPartition,
> > > > > > > int
> > > > > > > > leaderEpoch) and updates that as the log start offset." This
> > text
> > > > is
> > > > > > > still
> > > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > > >
> > > > > > > > 5115. There are still references to "remote log cleaners".
> > > > > > > >
> > > > > > > > 6000. Since we are returning new error codes, we need to bump
> > up
> > > > the
> > > > > > > > protocol version for Fetch request. Also, it will be useful to
> > > > > document
> > > > > > > all
> > > > > > > > new error codes and whether they are retriable or not.
> > > > > > > >
> > > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > > > > > leaderEpoch
> > > > > > > > is int32 instead of long.
> > > > > > > >
> > > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed
> > given
> > > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > > >
> > > > > > > > 6003. RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > topicPartition, long offset, int epochForOffset): Should this
> > > > return
> > > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > > >
> > > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to
> > pick a
> > > > > more
> > > > > > > > indicative name so that people understand what epoch this is.
> > > > > > > >
> > > > > > > > 6005. RemoteLogState: It seems it's better to split it between
> > > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since
> > the
> > > > > states
> > > > > > > > are never shared between the two use cases.
> > > > > > > >
> > > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> > However,
> > > > > is it
> > > > > > > > ture that other than the metadata topic, RLMM just needs to
> > know
> > > > > whether
> > > > > > > > there is a replica assigned to this broker and doesn't need to
> > know
> > > > > > > whether
> > > > > > > > the replica is the leader or the follower?
> > > > > > > >
> > > > > > > > 6007: "Handle expired remote segments (leader and follower)":
> > Why
> > > > is
> > > > > this
> > > > > > > > needed in both the leader and the follower?
> > > > > > > >
> > > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > > >                 "type": "int64",
> > > > > > > > The segment size can just be int32.
> > > > > > > >
> > > > > > > > 6009. For the record format in the log, it seems that we need
> > to
> > > > add
> > > > > > > record
> > > > > > > > type and record version before the serialized bytes. We can
> > follow
> > > > > the
> > > > > > > > convention used in
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > > .
> > > > > > > >
> > > > > > > > 6010. remote.log.manager.thread.pool.size: The default value
> > is 10.
> > > > > This
> > > > > > > > might be too high when enabling the tiered feature for the
> > first
> > > > > time.
> > > > > > > > Since there are lots of segments that need to be tiered
> > initially,
> > > > a
> > > > > > > large
> > > > > > > > number of threads could overwhelm the broker.
> > > > > > > >
> > > > > > > > 6011. "The number of milli seconds to keep the local log
> > segment
> > > > > before
> > > > > > > it
> > > > > > > > gets deleted. If not set, the value in `log.retention.minutes`
> > is
> > > > > used.
> > > > > > > If
> > > > > > > > set to -1, no time limit is applied." We should use
> > > > log.retention.ms
> > > > > > > > instead of log.retention.minutes.
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > > satish.duggana@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi,
> > > > > > > > > We updated the KIP with the points mentioned in the earlier
> > mail
> > > > > > > > > except for KIP-516 related changes. You can go through them
> > and
> > > > > let us
> > > > > > > > > know if you have any comments. We will update the KIP with
> > the
> > > > > > > > > remaining todo items and KIP-516 related changes by end of
> > this
> > > > > > > > > week(5th Dec).
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Satish.
> > > > > > > > >
> > > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > > satish.duggana@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > Hi Jun,
> > > > > > > > > > Thanks for your comments. Please find the inline replies
> > below.
> > > > > > > > > >
> > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting the
> > leader
> > > > > epoch
> > > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > > > mentioned an
> > > > > > > > > issue
> > > > > > > > > > earlier. Suppose the leader's local start offset is 100.
> > The
> > > > > follower
> > > > > > > > > finds
> > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > producerState
> > > > > > > with
> > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > producerState to
> > > > > > > > > > offset 100 requires more work since one needs to download
> > the
> > > > > > > previous
> > > > > > > > > > producerState up to offset 80 and then replay the messages
> > from
> > > > > 80 to
> > > > > > > > > 100.
> > > > > > > > > > It seems that it's simpler in this case for the follower
> > just
> > > > to
> > > > > > > take the
> > > > > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > > > > >
> > > > > > > > > > We chose that approach to avoid any edge cases here. It
> > may be
> > > > > > > > > > possible that the remote log segment that is received may
> > not
> > > > > have
> > > > > > > the
> > > > > > > > > > same leader epoch sequence from 100-120 as it contains on
> > the
> > > > > > > > > > leader(this can happen due to unclean leader). It is safe
> > to
> > > > > start
> > > > > > > > > > from what the leader returns here.Another way is to find
> > the
> > > > > remote
> > > > > > > > > > log segment
> > > > > > > > > >
> > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > > > > replicas
> > > > > > > for a
> > > > > > > > > > partition, not on the replicas for the
> > > > > __remote_log_segment_metadata
> > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > __remote_log_segment_metadata
> > > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > > >
> > > > > > > > > > RLMM will always receive the callback for the remote log
> > > > metadata
> > > > > > > > > > topic partitions hosted on the local broker and these will
> > be
> > > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > > >
> > > > > > > > > > 5100. KIP-516 has been accepted and is being implemented
> > now.
> > > > > Could
> > > > > > > you
> > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > >
> > > > > > > > > > We mentioned KIP-516 and how it helps. We will update this
> > KIP
> > > > > with
> > > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > > >
> > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > following two
> > > > > APIs
> > > > > > > are
> > > > > > > > > > used. According to the wiki, the former is used for topic
> > > > > deletion
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > latter is used for retention. It seems that retention
> > should
> > > > use
> > > > > the
> > > > > > > > > former
> > > > > > > > > > since remote segments without a matching epoch in the
> > leader
> > > > > > > (potentially
> > > > > > > > > > due to unclean leader election) also need to be garbage
> > > > > collected.
> > > > > > > The
> > > > > > > > > > latter seems to be used for the new leader to determine the
> > > > last
> > > > > > > tiered
> > > > > > > > > > segment.
> > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > >
> > > > > > > > > > Right,.that is what we are currently doing. We will update
> > the
> > > > > > > > > > javadocs and wiki with that. Earlier, we did not want to
> > remove
> > > > > the
> > > > > > > > > > segments which are not matched with leader epochs from the
> > > > ladder
> > > > > > > > > > partition as they may be used later by a replica which can
> > > > > become a
> > > > > > > > > > leader (unclean leader election) and refer those segments.
> > But
> > > > > that
> > > > > > > > > > may leak these segments in remote storage until the topic
> > > > > lifetime.
> > > > > > > We
> > > > > > > > > > decided to cleanup the segments with the oldest incase of
> > size
> > > > > based
> > > > > > > > > > retention also.
> > > > > > > > > >
> > > > > > > > > > 5102. RSM:
> > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems
> > that
> > > > > they can
> > > > > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > > > > >
> > > > > > > > > > It will be useful to have metadata for RSM to fetch log
> > > > segment.
> > > > > It
> > > > > > > > > > may create location/path using id with other metadata too.
> > > > > > > > > >
> > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > instead of
> > > > > Long?
> > > > > > > > > >
> > > > > > > > > > Wanted to keep endPosition as optional to read till the
> > end of
> > > > > the
> > > > > > > > > > segment and avoid sentinels.
> > > > > > > > > >
> > > > > > > > > > 5102.3 Why only some of the methods have default
> > implementation
> > > > > and
> > > > > > > > > others
> > > > > > > > > > Don't?
> > > > > > > > > >
> > > > > > > > > > Actually,  RSM will not have any default implementations.
> > > > Those 3
> > > > > > > > > > methods were made default earlier for tests etc. Updated
> > the
> > > > > wiki.
> > > > > > > > > >
> > > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > >
> > > > > > > > > > Sure, they will be added.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a
> > file
> > > > > since
> > > > > > > it
> > > > > > > > > will
> > > > > > > > > > be generated in memory.
> > > > > > > > > >
> > > > > > > > > > Right, this is in plan.
> > > > > > > > > >
> > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs
> > both
> > > > > > > baseOffset
> > > > > > > > > and
> > > > > > > > > > startOffset. For example, deleteRecords() could move the
> > > > > startOffset
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > middle of a segment. If we copy the full segment to remote
> > > > > storage,
> > > > > > > the
> > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > >
> > > > > > > > > > Good point. startOffset is baseOffset by default, if not
> > set
> > > > > > > explicitly.
> > > > > > > > > >
> > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > and LogSegmentData?
> > > > > > > > > >
> > > > > > > > > > Sure, updated the wiki.
> > > > > > > > > >
> > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > > inclusive/exclusive?
> > > > > > > > > >
> > > > > > > > > > It is inclusive, will update.
> > > > > > > > > >
> > > > > > > > > > 5103. configs:
> > > > > > > > > > 5103.1 Could we define the default value of non-required
> > > > configs
> > > > > > > (e.g the
> > > > > > > > > > size of new thread pools)?
> > > > > > > > > >
> > > > > > > > > > Sure, that makes sense.
> > > > > > > > > >
> > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > default to
> > > > > > > > > retention.ms,
> > > > > > > > > > instead of remote.log.retention.minutes. Similarly, it
> > seems
> > > > > > > > > > that local.log.retention.bytes should default to
> > segment.bytes.
> > > > > > > > > >
> > > > > > > > > > Right, we do not have  remote.log.retention as we discussed
> > > > > earlier.
> > > > > > > > > > Thanks for catching the typo.
> > > > > > > > > >
> > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description
> > > > says
> > > > > > > "used in
> > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > indexes and
> > > > > > > clean up
> > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > config remote.log.reader.threads for fetching remote data.
> > It's
> > > > > > > weird to
> > > > > > > > > > fetch remote index and log in different thread pools since
> > both
> > > > > are
> > > > > > > used
> > > > > > > > > > for serving fetch requests.
> > > > > > > > > >
> > > > > > > > > > Right, remote.log.manager.thread.pool is mainly used for
> > > > > copy/cleanup
> > > > > > > > > > activities. Fetch path always goes through
> > > > > remote.log.reader.threads.
> > > > > > > > > >
> > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> > amount
> > > > > of
> > > > > > > time
> > > > > > > > > to
> > > > > > > > > > back off when there is no work to do? If so, perhaps it
> > can be
> > > > > > > renamed as
> > > > > > > > > > backoff.ms.
> > > > > > > > > >
> > > > > > > > > > This is the delay interval for each iteration. It may be
> > > > renamed
> > > > > to
> > > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > > >
> > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > rlm_retry_interval_ms
> > > > > > > configs? If
> > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > >
> > > > > > > > > > remote.log.manager.task.interval.ms is the process
> > internal,
> > > > > retry
> > > > > > > > > > interval is missing in the configs, which will be updated
> > in
> > > > the
> > > > > KIP.
> > > > > > > > > >
> > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the
> > index
> > > > > > > files of
> > > > > > > > > > remote log segments to avoid multiple index fetches from
> > the
> > > > > remote
> > > > > > > > > > storage." Is the RLM in memory or on disk? If on disk,
> > where is
> > > > > it
> > > > > > > > > stored?
> > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > >
> > > > > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > > > > `remote-log-index-cache` under log dir. We plan to have a
> > > > config
> > > > > for
> > > > > > > > > > that instead of default. We will have a configuration for
> > that.
> > > > > > > > > >
> > > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest
> > Local
> > > > > Offset
> > > > > > > in
> > > > > > > > > > different places. It would be useful to standardize the
> > > > > terminology.
> > > > > > > > > >
> > > > > > > > > > Sure.
> > > > > > > > > >
> > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> > listed
> > > > > two
> > > > > > > > > options
> > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > > > > > >
> > > > > > > > > > 5107. Follower to leader transition: It has step 2, but not
> > > > step
> > > > > 1.
> > > > > > > > > > Step-1 is there but it is not explicitly highlighted. It is
> > > > > previous
> > > > > > > > > > table to step-2.
> > > > > > > > > >
> > > > > > > > > > 5108. If a consumer fetches from the remote data and the
> > remote
> > > > > > > storage
> > > > > > > > > is
> > > > > > > > > > not available, what error code is used in the fetch
> > response?
> > > > > > > > > >
> > > > > > > > > > Good point. We have not yet defined the error for this
> > case. We
> > > > > need
> > > > > > > > > > to define an error message and send the same in fetch
> > response.
> > > > > > > > > >
> > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the
> > first
> > > > > message
> > > > > > > > > > offset whose timestamp is >= to the given timestamp in the
> > > > > request.
> > > > > > > That
> > > > > > > > > > means it checks in remote log time indexes first, after
> > which
> > > > > local
> > > > > > > log
> > > > > > > > > > time indexes are checked." Could you document which method
> > in
> > > > > RLMM is
> > > > > > > > > used
> > > > > > > > > > for this?
> > > > > > > > > >
> > > > > > > > > > Okay.
> > > > > > > > > >
> > > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > metadata
> > > > > of
> > > > > > > that
> > > > > > > > > > partition with a delete marker and publishes them to RLMM."
> > > > This
> > > > > > > seems
> > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > >
> > > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > > >
> > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > earliest
> > > > > > > leader
> > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > > int
> > > > > > > > > > leaderEpoch) and updates that as the log start offset." Do
> > we
> > > > > need
> > > > > > > that
> > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > >
> > > > > > > > > > Good point. Right, existing replication protocol takes
> > care of
> > > > > > > > > > updating the followers’s log start offset received from the
> > > > > leader.
> > > > > > > > > >
> > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching
> > > > from
> > > > > > > remote
> > > > > > > > > > storage?
> > > > > > > > > >
> > > > > > > > > > Remote reads may fail within the current default wait
> > time, but
> > > > > > > > > > subsequent fetches would be able to serve as that data is
> > > > stored
> > > > > in
> > > > > > > > > > the local cache. This cache is currently implemented in
> > RSMs.
> > > > > But we
> > > > > > > > > > plan to pull this into the remote log messaging layer in
> > > > future.
> > > > > > > > > >
> > > > > > > > > > 5113. "Committed offsets can be stored in a local file to
> > avoid
> > > > > > > reading
> > > > > > > > > the
> > > > > > > > > > messages again when a broker is restarted." Could you
> > describe
> > > > > the
> > > > > > > format
> > > > > > > > > > and the location of the file? Also, could the same message
> > be
> > > > > > > processed
> > > > > > > > > by
> > > > > > > > > > RLMM again after broker restart? If so, how do we handle
> > that?
> > > > > > > > > >
> > > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > > >
> > > > > > > > > > 5114. Message format
> > > > > > > > > > 5114.1 There are two records named
> > > > RemoteLogSegmentMetadataRecord
> > > > > > > with
> > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > >
> > > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > > >
> > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > > > whether
> > > > > > > > > endOffset
> > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > It is inclusive, will update.
> > > > > > > > > >
> > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > > > LeaderEpoch
> > > > > > > a
> > > > > > > > > bit
> > > > > > > > > > more? Is that the epoch of the leader when it copies the
> > > > segment
> > > > > to
> > > > > > > > > remote
> > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > >
> > > > > > > > > > Right, this is the leader epoch of the broker which copied
> > this
> > > > > > > > > > segment. This is helpful in reason about which broker
> > copied
> > > > the
> > > > > > > > > > segment to remote storage.
> > > > > > > > > >
> > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more?
> > Each
> > > > > > > record in
> > > > > > > > > > Kafka already has a timestamp field. Could we just use
> > that?
> > > > > > > > > >
> > > > > > > > > > This is the  timestamp at which  the respective event
> > occurred.
> > > > > Added
> > > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > > > > > > > > implementation. We thought about that but it looked
> > cleaner to
> > > > > use at
> > > > > > > > > > the message structure level instead of getting that from
> > the
> > > > > consumer
> > > > > > > > > > record and using that to build the respective event.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > >
> > > > > > > > > > Right, it looks like config allows only int value >= 14.
> > > > > > > > > >
> > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with
> > the
> > > > log
> > > > > > > cleaner
> > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > >
> > > > > > > > > > I am fine with RemotePartitionRemover or
> > > > > RemoteLogDeletionManager(we
> > > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > > >
> > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > processes
> > > > it
> > > > > if
> > > > > > > it is
> > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > >
> > > > > > > > > > This is to handle duplicate delete_partition_marked
> > events. RLC
> > > > > > > > > > internally maintains a state for the delete_partition
> > events
> > > > and
> > > > > if
> > > > > > > it
> > > > > > > > > > already has an existing event then it ignores if it is
> > already
> > > > > being
> > > > > > > > > > processed.
> > > > > > > > > >
> > > > > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > > > > metadata
> > > > > > > > > topic?
> > > > > > > > > >
> > > > > > > > > > Right, this is in plan but did not mention it in the KIP.
> > This
> > > > > will
> > > > > > > be
> > > > > > > > > > useful for debugging purposes too.
> > > > > > > > > >
> > > > > > > > > > 5118. "Maximum remote log reader thread pool task queue
> > size.
> > > > If
> > > > > the
> > > > > > > task
> > > > > > > > > > queue is full, broker will stop reading remote log
> > segments."
> > > > > What
> > > > > > > do we
> > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > >
> > > > > > > > > > We return an error response for that partition.
> > > > > > > > > >
> > > > > > > > > > 5119. It would be useful to list all things not supported
> > in
> > > > the
> > > > > > > first
> > > > > > > > > > version in a Future work or Limitations section. For
> > example,
> > > > > > > compacted
> > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from true
> > to
> > > > > false,
> > > > > > > etc.
> > > > > > > > > >
> > > > > > > > > > We already have a non-goals section which is filled with
> > some
> > > > of
> > > > > > > these
> > > > > > > > > > details. Do we need another limitations section?
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Satish.
> > > > > > > > > >
> > > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io>
> > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > Hi, Satish,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > > > >
> > > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting the
> > > > leader
> > > > > > > epoch
> > > > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > > > mentioned
> > > > > > > an
> > > > > > > > > issue
> > > > > > > > > > > earlier. Suppose the leader's local start offset is 100.
> > The
> > > > > > > follower
> > > > > > > > > finds
> > > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > > producerState
> > > > > > > > > with
> > > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > > producerState
> > > > > > > to
> > > > > > > > > > > offset 100 requires more work since one needs to
> > download the
> > > > > > > previous
> > > > > > > > > > > producerState up to offset 80 and then replay the
> > messages
> > > > > from 80
> > > > > > > to
> > > > > > > > > 100.
> > > > > > > > > > > It seems that it's simpler in this case for the follower
> > just
> > > > > to
> > > > > > > take
> > > > > > > > > the
> > > > > > > > > > > remote segment as it is and start fetching from offset
> > 120.
> > > > > > > > > > >
> > > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > > > > replicas
> > > > > > > for
> > > > > > > > > a
> > > > > > > > > > > partition, not on the replicas for the
> > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > > > >
> > > > > > > > > > > 5100. KIP-516 has been accepted and is being implemented
> > now.
> > > > > > > Could you
> > > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > > >
> > > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> > following
> > > > two
> > > > > > > APIs
> > > > > > > > > are
> > > > > > > > > > > used. According to the wiki, the former is used for topic
> > > > > deletion
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > latter is used for retention. It seems that retention
> > should
> > > > > use
> > > > > > > the
> > > > > > > > > former
> > > > > > > > > > > since remote segments without a matching epoch in the
> > leader
> > > > > > > > > (potentially
> > > > > > > > > > > due to unclean leader election) also need to be garbage
> > > > > collected.
> > > > > > > The
> > > > > > > > > > > latter seems to be used for the new leader to determine
> > the
> > > > > last
> > > > > > > tiered
> > > > > > > > > > > segment.
> > > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > > >
> > > > > > > > > > > 5102. RSM:
> > > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems
> > that
> > > > > they
> > > > > > > can
> > > > > > > > > > > use RemoteLogSegmentId instead of
> > RemoteLogSegmentMetadata.
> > > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> > instead
> > > > of
> > > > > > > Long?
> > > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > > implementation and
> > > > > > > > > others
> > > > > > > > > > > don't?
> > > > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a
> > file
> > > > > > > since it
> > > > > > > > > will
> > > > > > > > > > > be generated in memory.
> > > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs
> > both
> > > > > > > > > baseOffset and
> > > > > > > > > > > startOffset. For example, deleteRecords() could move the
> > > > > > > startOffset
> > > > > > > > > to the
> > > > > > > > > > > middle of a segment. If we copy the full segment to
> > remote
> > > > > > > storage, the
> > > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > > and LogSegmentData?
> > > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > is
> > > > > > > > > > > inclusive/exclusive?
> > > > > > > > > > >
> > > > > > > > > > > 5103. configs:
> > > > > > > > > > > 5103.1 Could we define the default value of non-required
> > > > > configs
> > > > > > > (e.g
> > > > > > > > > the
> > > > > > > > > > > size of new thread pools)?
> > > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> > default
> > > > to
> > > > > > > > > retention.ms,
> > > > > > > > > > > instead of remote.log.retention.minutes. Similarly, it
> > seems
> > > > > > > > > > > that local.log.retention.bytes should default to
> > > > segment.bytes.
> > > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> > description
> > > > > says
> > > > > > > "used
> > > > > > > > > in
> > > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> > indexes
> > > > and
> > > > > > > clean
> > > > > > > > > up
> > > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > > config remote.log.reader.threads for fetching remote
> > data.
> > > > It's
> > > > > > > weird
> > > > > > > > > to
> > > > > > > > > > > fetch remote index and log in different thread pools
> > since
> > > > > both are
> > > > > > > > > used
> > > > > > > > > > > for serving fetch requests.
> > > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> > > > > amount of
> > > > > > > > > time to
> > > > > > > > > > > back off when there is no work to do? If so, perhaps it
> > can
> > > > be
> > > > > > > renamed
> > > > > > > > > as
> > > > > > > > > > > backoff.ms.
> > > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> > rlm_retry_interval_ms
> > > > > > > configs?
> > > > > > > > > If
> > > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > > >
> > > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the
> > > > index
> > > > > > > files
> > > > > > > > > of
> > > > > > > > > > > remote log segments to avoid multiple index fetches from
> > the
> > > > > remote
> > > > > > > > > > > storage." Is the RLM in memory or on disk? If on disk,
> > where
> > > > > is it
> > > > > > > > > stored?
> > > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > > >
> > > > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest
> > Local
> > > > > > > Offset in
> > > > > > > > > > > different places. It would be useful to standardize the
> > > > > > > terminology.
> > > > > > > > > > >
> > > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> > > > > listed two
> > > > > > > > > options
> > > > > > > > > > > without saying which option is chosen.
> > > > > > > > > > >
> > > > > > > > > > > 5107. Follower to leader transition: It has step 2, but
> > not
> > > > > step 1.
> > > > > > > > > > >
> > > > > > > > > > > 5108. If a consumer fetches from the remote data and the
> > > > remote
> > > > > > > > > storage is
> > > > > > > > > > > not available, what error code is used in the fetch
> > response?
> > > > > > > > > > >
> > > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the
> > first
> > > > > > > message
> > > > > > > > > > > offset whose timestamp is >= to the given timestamp in
> > the
> > > > > request.
> > > > > > > > > That
> > > > > > > > > > > means it checks in remote log time indexes first, after
> > which
> > > > > > > local log
> > > > > > > > > > > time indexes are checked." Could you document which
> > method in
> > > > > RLMM
> > > > > > > is
> > > > > > > > > used
> > > > > > > > > > > for this?
> > > > > > > > > > >
> > > > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > > > > metadata of
> > > > > > > that
> > > > > > > > > > > partition with a delete marker and publishes them to
> > RLMM."
> > > > > This
> > > > > > > seems
> > > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > > >
> > > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > > earliest
> > > > > > > leader
> > > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > > int
> > > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> > Do we
> > > > > need
> > > > > > > that
> > > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > > >
> > > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> > fetching
> > > > > from
> > > > > > > remote
> > > > > > > > > > > storage?
> > > > > > > > > > >
> > > > > > > > > > > 5113. "Committed offsets can be stored in a local file to
> > > > avoid
> > > > > > > > > reading the
> > > > > > > > > > > messages again when a broker is restarted." Could you
> > > > describe
> > > > > the
> > > > > > > > > format
> > > > > > > > > > > and the location of the file? Also, could the same
> > message be
> > > > > > > > > processed by
> > > > > > > > > > > RLMM again after broker restart? If so, how do we handle
> > > > that?
> > > > > > > > > > >
> > > > > > > > > > > 5114. Message format
> > > > > > > > > > > 5114.1 There are two records named
> > > > > RemoteLogSegmentMetadataRecord
> > > > > > > with
> > > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > > > > whether
> > > > > > > > > endOffset
> > > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > > > > > LeaderEpoch a
> > > > > > > > > bit
> > > > > > > > > > > more? Is that the epoch of the leader when it copies the
> > > > > segment to
> > > > > > > > > remote
> > > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more?
> > > > Each
> > > > > > > record
> > > > > > > > > in
> > > > > > > > > > > Kafka already has a timestamp field. Could we just use
> > that?
> > > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > > >
> > > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with
> > the
> > > > > log
> > > > > > > > > cleaner
> > > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > > >
> > > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> > processes
> > > > > it
> > > > > > > if it
> > > > > > > > > is
> > > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > > >
> > > > > > > > > > > 5117. Should we add a new MessageFormatter to read the
> > tier
> > > > > > > metadata
> > > > > > > > > topic?
> > > > > > > > > > >
> > > > > > > > > > > 5118. "Maximum remote log reader thread pool task queue
> > size.
> > > > > If
> > > > > > > the
> > > > > > > > > task
> > > > > > > > > > > queue is full, broker will stop reading remote log
> > segments."
> > > > > What
> > > > > > > do
> > > > > > > > > we
> > > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > > >
> > > > > > > > > > > 5119. It would be useful to list all things not
> > supported in
> > > > > the
> > > > > > > first
> > > > > > > > > > > version in a Future work or Limitations section. For
> > example,
> > > > > > > compacted
> > > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> > true to
> > > > > false,
> > > > > > > > > etc.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > > (
> > > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > >
> > > >
> >

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

Posted by Jun Rao <ju...@confluent.io>.
Hi, Satish,

Thanks for the reply. A few more followup comments.

6000. Since we are returning new error codes, we need to bump up the
protocol version for Fetch request. Also, it will be useful to document all
new error codes and whether they are retriable or not.
This previous comment doesn't seem to be addressed.

9110. flat_file_format: Could you define the serialized representation for
each field in the header?

9111. RPM has the following 2 steps.
"1. The controller publishes delete_partition_marked event to say that the
partition is marked for deletion. There can be multiple events published
when the controller restarts or failover and this event will be
deduplicated by RPM.
2. RPM receives the delete_partition_marked and processes it if it is not
yet processed earlier."
What triggers RPM to read __remote_log_metadata? Is RPM part of the default
RLMM implementation or is it meant for any RLMM implementation?

9112. remote.log.manager.task.retry.backoff.ms: It would be useful to make
it clear in the comment that this is for the initial retry backoff.

9113. RLMM:
9113.1 updateRemoteLogSegmentMetadata(): This comment needs to be updated.
9113.2 Should RemoteLogSegmentMetadataUpdate include a leadeEpoch field
since all other update events have leaderEpoch?
9113.3 Could we rename RemotePartitionState to RemotePartitionDeleteState
to make it clear this is for deletion?
9113.4 Could we rename RemoteLogState to RemoteLogSegmentState to make it
clear this is for segment?

9114.Upgrade:
9114.1 It seems that we require message format > 0.11 for turning on the
remote store feature.
9114.2 It's not clear to me why remote.log.storage.system.enable needs to
be set to true before bumping up inter.broker.protocol.version.
9114.3 "If the topic-id is not received in the LeaderAndIsr request then
remote log storage will not start. But it will log an error message in the
log. One way to address this is to do a rolling restart of that broker, so
that the leader will be moved to another broker and the controller will
send LeaderAndIsr with the registered topic-id." Why is this needed? With
KIP-516, after upgrading to the latest protocol, topicIds are auto created.

9115. testing: Could you comment on how we plan to do integration and
system tests? Do we plan to include, for example, an in-memory
implementation of RSM?

9116. There is still a todo for the message formatter.

Jun

On Sat, Jan 9, 2021 at 2:04 AM Satish Duggana <sa...@gmail.com>
wrote:

> Hi Jun,
> Thanks for your comments. Please find the inline replies below.
>
> 6022. For packages used for server plugins, the convention is to
> use org.apache.kafka.server. See java-based Authorizer as an example.
>
> Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
> ‘org.apache.kafka.server.log.remote.storage’.  Updated in the KIP.
>
> 9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat
> file captures the state of the remote segments. After a partition is
> deleted, it seems that we just need to remove the partitions's remote
> segments from the flat file.
>
> DeletePartitionState might not yet have been processed by RPM and not
> completed. We  will not have  that in flat  file format  once it
> reaches DELETE_PARTITION_FINISHED state.
>
> 9101. Upgrade: It will be useful to allow direct upgrade from an old
> version. It seems that's doable. One can just do the normal upgrade first
> and wait enough time (for producer snapshots to be built), and then enable
> remote storage.
>
> Upgrade notes updates in the KIP.
>
> 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> tracking the remote segments when RLMM.onPartitionLeadershipChanges() is
> called with the broker being the leader for __remote_log_metadata
> partition? If so, could we document it?
>
> The current plan is to have that as part of RLMM and RPM uses that to
> get the remote segments list. I will add this detail in the respective
> sections.
>
> Satish.
>
> On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
> >
> > Hi, Satish,
> >
> > Thanks for the reply. A few more followup comments.
> >
> > 6022. For packages used for server plugins, the convention is to
> > use org.apache.kafka.server. See java-based Authorizer as an example.
> >
> > 9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat
> > file captures the state of the remote segments. After a partition is
> > deleted, it seems that we just need to remove the partitions's remote
> > segments from the flat file.
> >
> > 9101. Upgrade: It will be useful to allow direct upgrade from an old
> > version. It seems that's doable. One can just do the normal upgrade first
> > and wait enough time (for producer snapshots to be built), and then
> enable
> > remote storage.
> >
> > 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> > tracking the remote segments when RLMM.onPartitionLeadershipChanges() is
> > called with the broker being the leader for __remote_log_metadata
> > partition? If so, could we document it?
> >
> > Jun
> >
> > On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <kprakasam@confluent.io
> >
> > wrote:
> >
> > > Hi Satish,
> > >
> > > Thanks for the updates! A few more comments below.
> > >
> > > 9001. Under the "Upgrade" section, there is a line mentioning:
> "Upgrade the
> > > existing Kafka cluster to 2.7 version and allow this to run for the log
> > > retention of user topics that you want to enable tiered storage. This
> will
> > > allow all the topics to have the producer snapshots generated for each
> log
> > > segment." -- Which associated change in AK were you referring to here?
> Is
> > > it: https://github.com/apache/kafka/pull/7929 ? It seems like I don't
> see
> > > it in the 2.7 release branch yet, here is the link:
> > > https://github.com/apache/kafka/commits/2.7.
> > >
> > > 9002. Under the "Upgrade" section, the configuration mentioned is
> > > 'remote.log.storage.system.enable'. However, under "Public Interfaces"
> > > section the corresponding configuration is
> 'remote.storage.system.enable'.
> > > Could we use the same one in both, maybe
> > > 'remote.log.storage.system.enable'?
> > >
> > > 9003. Under "Per Topic Configuration", the KIP recommends setting
> > > 'remote.log.storage.enable' to true at a per-topic level. It will be
> useful
> > > to add a line that if the user wants to enable it for all topics, then
> they
> > > should be able to set the cluster-wide default to true. Also, it will
> be
> > > useful to mention that the KIP currently does not support setting it to
> > > false (after it is set to true), and add that to the future work
> section.
> > >
> > > 9004. Under "Committed offsets file format", the sample provided shows
> > > partition number and offset. Is the topic name required for identifying
> > > which topic the partitions belong to?
> > >
> > > 9005. Under "Internal flat-file store format of remote log metadata",
> it
> > > seems useful to specify both topic name and topic ID for debugging
> > > purposes.
> > >
> > > 9006. Under "Internal flat-file store format of remote log metadata",
> the
> > > description of "metadata-topic-offset" currently says "offset of the
> remote
> > > log metadata topic from which this topic partition's remote log
> metadata is
> > > fetched." Just for the wording, perhaps you meant to refer to the
> offset
> > > upto which the file has been committed? i.e. "offset of the remote log
> > > metadata topic upto which this topic partition's remote log metadata
> has
> > > been committed into this file."
> > >
> > > 9007. Under "Internal flat-file store format of remote log metadata",
> the
> > > schema of the payload (i.e. beyond the header) seems to contain the
> events
> > > from the metadata topic. It seems useful to instead persist the
> > > representation of the materialized state of the events, so that for the
> > > same segment only the latest state is stored. Besides reducing storage
> > > footprint, this also is likely to relate directly with the in-memory
> > > representation of the RLMM cache (which probably is some kind of a Map
> with
> > > key being segment ID and value being the segment state), so recovery
> from
> > > disk will be straightforward.
> > >
> > > 9008. Under "Topic deletion lifecycle", step (1), it will be useful to
> > > mention when in the deletion flow does the controller publish the
> > > delete_partition_marked event to say that the partition is marked for
> > > deletion?
> > >
> > > 9009. There are ~4 TODOs in the KIP. Could you please address these or
> > > remove them?
> > >
> > > 9010. There is a reference to a Google doc on the KIP which was used
> > > earlier for discussions. Please could you remove the reference, since
> the
> > > KIP is the source of the truth?
> > >
> > > 9011. This feedback is from an earlier comment. 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. What do you think?
> > >
> > >
> > > Cheers,
> > > Kowshik
> > >
> > >
> > > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <
> satish.duggana@gmail.com>
> > > wrote:
> > >
> > > > Hi Jun,
> > > > Thanks for your comments. Please go through the inline replies.
> > > >
> > > >
> > > > 5102.2: It seems that both positions can just be int. Another option
> is
> > > to
> > > > have two methods. Would it be clearer?
> > > >
> > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > remoteLogSegmentMetadata,  int startPosition)
> > > throwsRemoteStorageException;
> > > >
> > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > remoteLogSegmentMetadata, int startPosition, int endPosition) throws
> > > > RemoteStorageException;
> > > >
> > > > That makes sense to me, updated the KIP.
> > > >
> > > > 6003: Could you also update the javadoc for the return value?
> > > >
> > > > Updated.
> > > >
> > > > 6020: local.log.retention.bytes: Should it default to
> log.retention.bytes
> > > > to be consistent with local.log.retention.ms?
> > > >
> > > > Yes, it can be defaulted to log.retention.bytes.
> > > >
> > > > 6021: Could you define TopicIdPartition?
> > > >
> > > > Added TopicIdPartition in the KIP.
> > > >
> > > > 6022: For all public facing classes, could you specify the package
> name?
> > > >
> > > > Updated.
> > > >
> > > >
> > > > Thanks,
> > > > Satish.
> > > >
> > > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the reply. A few more comments below.
> > > > >
> > > > > 5102.2: It seems that both positions can just be int. Another
> option is
> > > > to
> > > > > have two methods. Would it be clearer?
> > > > >
> > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata,
> > > > >                                     int startPosition) throws
> > > > > RemoteStorageException;
> > > > >
> > > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > > remoteLogSegmentMetadata,
> > > > >                                     int startPosition, int
> endPosition)
> > > > > throws RemoteStorageException;
> > > > >
> > > > > 6003: Could you also update the javadoc for the return value?
> > > > >
> > > > > 6010: What kind of tiering throughput have you seen with 5 threads?
> > > > >
> > > > > 6020: local.log.retention.bytes: Should it default to
> > > log.retention.bytes
> > > > > to be consistent with local.log.retention.ms?
> > > > >
> > > > > 6021: Could you define TopicIdPartition?
> > > > >
> > > > > 6022: For all public facing classes, could you specify the package
> > > name?
> > > > >
> > > > > It seems that you already added the topicId support. Two other
> > > remaining
> > > > > items are (a) the format of local tier metadata storage and (b)
> > > upgrade.
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > > satish.duggana@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Jun,
> > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > >
> > > > > > >605.2 It's rare for the follower to need the remote data. So,
> the
> > > > current
> > > > > > approach is fine too. Could you document the process of
> rebuilding
> > > the
> > > > > > producer state since we can't simply trim the producerState to an
> > > > offset in
> > > > > > the middle of a segment.
> > > > > >
> > > > > > Will clarify in the KIP.
> > > > > >
> > > > > > >5102.2 Would it be clearer to make startPosiont long and
> endPosition
> > > > of
> > > > > > Optional<Long>?
> > > > > >
> > > > > > We will have arg checks with respective validation. It is not a
> good
> > > > > > practice to have arguments with optional as mentioned here.
> > > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > > >
> > > > > >
> > > > > > >5102.5 LogSegmentData still has leaderEpochIndex as File
> instead of
> > > > > > ByteBuffer.
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > > >5102.7 Could you define all public methods for LogSegmentData?
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > > >5103.5 Could you change the reference to
> rlm_process_interval_ms and
> > > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > > interval
> > > > > > config seems still missing. It would be useful to support
> exponential
> > > > > > backoff with the retry interval config.
> > > > > >
> > > > > > Good point. We wanted the retry with truncated exponential
> backoff,
> > > > > > updated the KIP.
> > > > > >
> > > > > > >5111. "RLM follower fetches the earliest offset for the earliest
> > > > leader
> > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > topicPartition,
> > > > int
> > > > > > leaderEpoch) and updates that as the log start offset." This
> text is
> > > > still
> > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > > >5115. There are still references to "remote log cleaners".
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > > >6000. Since we are returning new error codes, we need to bump
> up the
> > > > > > protocol version for Fetch request. Also, it will be useful to
> > > > document all
> > > > > > new error codes and whether they are retriable or not.
> > > > > >
> > > > > > Sure, we will add that in the KIP.
> > > > > >
> > > > > > >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > > leaderEpoch
> > > > > > is int32 instead of long.
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed
> given
> > > > > > RemoteLogSegmentMetadata.state()?
> > > > > >
> > > > > > No, it is fixed.
> > > > > >
> > > > > > >6003. RemoteLogSegmentMetadata
> > > remoteLogSegmentMetadata(TopicPartition
> > > > > > topicPartition, long offset, int epochForOffset): Should this
> return
> > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > >
> > > > > > That makes sense, updated.
> > > > > >
> > > > > > >6005. RemoteLogState: It seems it's better to split it between
> > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since
> the
> > > > states
> > > > > > are never shared between the two use cases.
> > > > > >
> > > > > > Agree with that, updated.
> > > > > >
> > > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> However,
> > > > is it
> > > > > > ture that other than the metadata topic, RLMM just needs to know
> > > > whether
> > > > > > there is a replica assigned to this broker and doesn't need to
> know
> > > > whether
> > > > > > the replica is the leader or the follower?
> > > > > >
> > > > > > That may be true. If the implementation does not need that, it
> can
> > > > > > ignore the information in the callback.
> > > > > >
> > > > > > >6007: "Handle expired remote segments (leader and follower)":
> Why is
> > > > this
> > > > > > needed in both the leader and the follower?
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > > >                 "type": "int64",
> > > > > > The segment size can just be int32.
> > > > > >
> > > > > > Updated.
> > > > > >
> > > > > > >6009. For the record format in the log, it seems that we need
> to add
> > > > > > record
> > > > > > type and record version before the serialized bytes. We can
> follow
> > > the
> > > > > > convention used in
> > > > > >
> > > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > >
> > > > > > Yes, KIP already mentions that these are serialized before the
> > > payload
> > > > > > as below. We will mention explicitly that these two are written
> > > before
> > > > > > the data is written.
> > > > > >
> > > > > > RLMM instance on broker publishes the message to the topic with
> key
> > > as
> > > > > > null and value with the below format.
> > > > > >
> > > > > > type      : unsigned var int, represents the value type. This
> value
> > > is
> > > > > > 'apikey' as mentioned in the schema.
> > > > > > version : unsigned var int, the 'version' number of the type as
> > > > > > mentioned in the schema.
> > > > > > data      : record payload in kafka protocol message format.
> > > > > >
> > > > > >
> > > > > > >6010. remote.log.manager.thread.pool.size: The default value is
> 10.
> > > > This
> > > > > > might be too high when enabling the tiered feature for the first
> > > time.
> > > > > > Since there are lots of segments that need to be tiered
> initially, a
> > > > large
> > > > > > number of threads could overwhelm the broker.
> > > > > >
> > > > > > Is the default value 5 reasonable?
> > > > > >
> > > > > > 6011. "The number of milli seconds to keep the local log segment
> > > > before it
> > > > > > gets deleted. If not set, the value in `log.retention.minutes` is
> > > > used. If
> > > > > > set to -1, no time limit is applied." We should use
> log.retention.ms
> > > > > > instead of log.retention.minutes.
> > > > > > Nice typo catch. Updated the KIP.
> > > > > >
> > > > > > Thanks,
> > > > > > Satish.
> > > > > >
> > > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > > >
> > > > > > > Hi, Satish,
> > > > > > >
> > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > >
> > > > > > > 605.2 It's rare for the follower to need the remote data. So,
> the
> > > > current
> > > > > > > approach is fine too. Could you document the process of
> rebuilding
> > > > the
> > > > > > > producer state since we can't simply trim the producerState to
> an
> > > > offset
> > > > > > in
> > > > > > > the middle of a segment.
> > > > > > >
> > > > > > > 5102.2 Would it be clearer to make startPosiont long and
> > > endPosition
> > > > of
> > > > > > > Optional<Long>?
> > > > > > >
> > > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File
> instead of
> > > > > > > ByteBuffer.
> > > > > > >
> > > > > > > 5102.7 Could you define all public methods for LogSegmentData?
> > > > > > >
> > > > > > > 5103.5 Could you change the reference to
> rlm_process_interval_ms
> > > and
> > > > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > > > interval
> > > > > > > config seems still missing. It would be useful to support
> > > exponential
> > > > > > > backoff with the retry interval config.
> > > > > > >
> > > > > > > 5111. "RLM follower fetches the earliest offset for the
> earliest
> > > > leader
> > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > topicPartition,
> > > > > > int
> > > > > > > leaderEpoch) and updates that as the log start offset." This
> text
> > > is
> > > > > > still
> > > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > > >
> > > > > > > 5115. There are still references to "remote log cleaners".
> > > > > > >
> > > > > > > 6000. Since we are returning new error codes, we need to bump
> up
> > > the
> > > > > > > protocol version for Fetch request. Also, it will be useful to
> > > > document
> > > > > > all
> > > > > > > new error codes and whether they are retriable or not.
> > > > > > >
> > > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > > > > leaderEpoch
> > > > > > > is int32 instead of long.
> > > > > > >
> > > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed
> given
> > > > > > > RemoteLogSegmentMetadata.state()?
> > > > > > >
> > > > > > > 6003. RemoteLogSegmentMetadata
> > > > remoteLogSegmentMetadata(TopicPartition
> > > > > > > topicPartition, long offset, int epochForOffset): Should this
> > > return
> > > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > > >
> > > > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to
> pick a
> > > > more
> > > > > > > indicative name so that people understand what epoch this is.
> > > > > > >
> > > > > > > 6005. RemoteLogState: It seems it's better to split it between
> > > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since
> the
> > > > states
> > > > > > > are never shared between the two use cases.
> > > > > > >
> > > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok.
> However,
> > > > is it
> > > > > > > ture that other than the metadata topic, RLMM just needs to
> know
> > > > whether
> > > > > > > there is a replica assigned to this broker and doesn't need to
> know
> > > > > > whether
> > > > > > > the replica is the leader or the follower?
> > > > > > >
> > > > > > > 6007: "Handle expired remote segments (leader and follower)":
> Why
> > > is
> > > > this
> > > > > > > needed in both the leader and the follower?
> > > > > > >
> > > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > > >                 "type": "int64",
> > > > > > > The segment size can just be int32.
> > > > > > >
> > > > > > > 6009. For the record format in the log, it seems that we need
> to
> > > add
> > > > > > record
> > > > > > > type and record version before the serialized bytes. We can
> follow
> > > > the
> > > > > > > convention used in
> > > > > > >
> > > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > > .
> > > > > > >
> > > > > > > 6010. remote.log.manager.thread.pool.size: The default value
> is 10.
> > > > This
> > > > > > > might be too high when enabling the tiered feature for the
> first
> > > > time.
> > > > > > > Since there are lots of segments that need to be tiered
> initially,
> > > a
> > > > > > large
> > > > > > > number of threads could overwhelm the broker.
> > > > > > >
> > > > > > > 6011. "The number of milli seconds to keep the local log
> segment
> > > > before
> > > > > > it
> > > > > > > gets deleted. If not set, the value in `log.retention.minutes`
> is
> > > > used.
> > > > > > If
> > > > > > > set to -1, no time limit is applied." We should use
> > > log.retention.ms
> > > > > > > instead of log.retention.minutes.
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi,
> > > > > > > > We updated the KIP with the points mentioned in the earlier
> mail
> > > > > > > > except for KIP-516 related changes. You can go through them
> and
> > > > let us
> > > > > > > > know if you have any comments. We will update the KIP with
> the
> > > > > > > > remaining todo items and KIP-516 related changes by end of
> this
> > > > > > > > week(5th Dec).
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Satish.
> > > > > > > >
> > > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > > satish.duggana@gmail.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > Hi Jun,
> > > > > > > > > Thanks for your comments. Please find the inline replies
> below.
> > > > > > > > >
> > > > > > > > > 605.2 "Build the local leader epoch cache by cutting the
> leader
> > > > epoch
> > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > > mentioned an
> > > > > > > > issue
> > > > > > > > > earlier. Suppose the leader's local start offset is 100.
> The
> > > > follower
> > > > > > > > finds
> > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > producerState
> > > > > > with
> > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > producerState to
> > > > > > > > > offset 100 requires more work since one needs to download
> the
> > > > > > previous
> > > > > > > > > producerState up to offset 80 and then replay the messages
> from
> > > > 80 to
> > > > > > > > 100.
> > > > > > > > > It seems that it's simpler in this case for the follower
> just
> > > to
> > > > > > take the
> > > > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > > > >
> > > > > > > > > We chose that approach to avoid any edge cases here. It
> may be
> > > > > > > > > possible that the remote log segment that is received may
> not
> > > > have
> > > > > > the
> > > > > > > > > same leader epoch sequence from 100-120 as it contains on
> the
> > > > > > > > > leader(this can happen due to unclean leader). It is safe
> to
> > > > start
> > > > > > > > > from what the leader returns here.Another way is to find
> the
> > > > remote
> > > > > > > > > log segment
> > > > > > > > >
> > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > > > replicas
> > > > > > for a
> > > > > > > > > partition, not on the replicas for the
> > > > __remote_log_segment_metadata
> > > > > > > > > partition. It's not clear how the leader of
> > > > > > __remote_log_segment_metadata
> > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > >
> > > > > > > > > RLMM will always receive the callback for the remote log
> > > metadata
> > > > > > > > > topic partitions hosted on the local broker and these will
> be
> > > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > > >
> > > > > > > > > 5100. KIP-516 has been accepted and is being implemented
> now.
> > > > Could
> > > > > > you
> > > > > > > > > update the KIP based on topicID?
> > > > > > > > >
> > > > > > > > > We mentioned KIP-516 and how it helps. We will update this
> KIP
> > > > with
> > > > > > > > > all the changes it brings with KIP-516.
> > > > > > > > >
> > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> following two
> > > > APIs
> > > > > > are
> > > > > > > > > used. According to the wiki, the former is used for topic
> > > > deletion
> > > > > > and
> > > > > > > > the
> > > > > > > > > latter is used for retention. It seems that retention
> should
> > > use
> > > > the
> > > > > > > > former
> > > > > > > > > since remote segments without a matching epoch in the
> leader
> > > > > > (potentially
> > > > > > > > > due to unclean leader election) also need to be garbage
> > > > collected.
> > > > > > The
> > > > > > > > > latter seems to be used for the new leader to determine the
> > > last
> > > > > > tiered
> > > > > > > > > segment.
> > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > >
> > > > > > > > > Right,.that is what we are currently doing. We will update
> the
> > > > > > > > > javadocs and wiki with that. Earlier, we did not want to
> remove
> > > > the
> > > > > > > > > segments which are not matched with leader epochs from the
> > > ladder
> > > > > > > > > partition as they may be used later by a replica which can
> > > > become a
> > > > > > > > > leader (unclean leader election) and refer those segments.
> But
> > > > that
> > > > > > > > > may leak these segments in remote storage until the topic
> > > > lifetime.
> > > > > > We
> > > > > > > > > decided to cleanup the segments with the oldest incase of
> size
> > > > based
> > > > > > > > > retention also.
> > > > > > > > >
> > > > > > > > > 5102. RSM:
> > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems
> that
> > > > they can
> > > > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > > > >
> > > > > > > > > It will be useful to have metadata for RSM to fetch log
> > > segment.
> > > > It
> > > > > > > > > may create location/path using id with other metadata too.
> > > > > > > > >
> > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> instead of
> > > > Long?
> > > > > > > > >
> > > > > > > > > Wanted to keep endPosition as optional to read till the
> end of
> > > > the
> > > > > > > > > segment and avoid sentinels.
> > > > > > > > >
> > > > > > > > > 5102.3 Why only some of the methods have default
> implementation
> > > > and
> > > > > > > > others
> > > > > > > > > Don't?
> > > > > > > > >
> > > > > > > > > Actually,  RSM will not have any default implementations.
> > > Those 3
> > > > > > > > > methods were made default earlier for tests etc. Updated
> the
> > > > wiki.
> > > > > > > > >
> > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > >
> > > > > > > > > Sure, they will be added.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a
> file
> > > > since
> > > > > > it
> > > > > > > > will
> > > > > > > > > be generated in memory.
> > > > > > > > >
> > > > > > > > > Right, this is in plan.
> > > > > > > > >
> > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs
> both
> > > > > > baseOffset
> > > > > > > > and
> > > > > > > > > startOffset. For example, deleteRecords() could move the
> > > > startOffset
> > > > > > to
> > > > > > > > the
> > > > > > > > > middle of a segment. If we copy the full segment to remote
> > > > storage,
> > > > > > the
> > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > >
> > > > > > > > > Good point. startOffset is baseOffset by default, if not
> set
> > > > > > explicitly.
> > > > > > > > >
> > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > and LogSegmentData?
> > > > > > > > >
> > > > > > > > > Sure, updated the wiki.
> > > > > > > > >
> > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > RemoteLogSegmentMetadata is
> > > > > > > > > inclusive/exclusive?
> > > > > > > > >
> > > > > > > > > It is inclusive, will update.
> > > > > > > > >
> > > > > > > > > 5103. configs:
> > > > > > > > > 5103.1 Could we define the default value of non-required
> > > configs
> > > > > > (e.g the
> > > > > > > > > size of new thread pools)?
> > > > > > > > >
> > > > > > > > > Sure, that makes sense.
> > > > > > > > >
> > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> default to
> > > > > > > > retention.ms,
> > > > > > > > > instead of remote.log.retention.minutes. Similarly, it
> seems
> > > > > > > > > that local.log.retention.bytes should default to
> segment.bytes.
> > > > > > > > >
> > > > > > > > > Right, we do not have  remote.log.retention as we discussed
> > > > earlier.
> > > > > > > > > Thanks for catching the typo.
> > > > > > > > >
> > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description
> > > says
> > > > > > "used in
> > > > > > > > > scheduling tasks to copy segments, fetch remote log
> indexes and
> > > > > > clean up
> > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > config remote.log.reader.threads for fetching remote data.
> It's
> > > > > > weird to
> > > > > > > > > fetch remote index and log in different thread pools since
> both
> > > > are
> > > > > > used
> > > > > > > > > for serving fetch requests.
> > > > > > > > >
> > > > > > > > > Right, remote.log.manager.thread.pool is mainly used for
> > > > copy/cleanup
> > > > > > > > > activities. Fetch path always goes through
> > > > remote.log.reader.threads.
> > > > > > > > >
> > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> amount
> > > > of
> > > > > > time
> > > > > > > > to
> > > > > > > > > back off when there is no work to do? If so, perhaps it
> can be
> > > > > > renamed as
> > > > > > > > > backoff.ms.
> > > > > > > > >
> > > > > > > > > This is the delay interval for each iteration. It may be
> > > renamed
> > > > to
> > > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > > >
> > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> rlm_retry_interval_ms
> > > > > > configs? If
> > > > > > > > > so, they need to be listed in this section.
> > > > > > > > >
> > > > > > > > > remote.log.manager.task.interval.ms is the process
> internal,
> > > > retry
> > > > > > > > > interval is missing in the configs, which will be updated
> in
> > > the
> > > > KIP.
> > > > > > > > >
> > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the
> index
> > > > > > files of
> > > > > > > > > remote log segments to avoid multiple index fetches from
> the
> > > > remote
> > > > > > > > > storage." Is the RLM in memory or on disk? If on disk,
> where is
> > > > it
> > > > > > > > stored?
> > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > >
> > > > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > > > `remote-log-index-cache` under log dir. We plan to have a
> > > config
> > > > for
> > > > > > > > > that instead of default. We will have a configuration for
> that.
> > > > > > > > >
> > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest
> Local
> > > > Offset
> > > > > > in
> > > > > > > > > different places. It would be useful to standardize the
> > > > terminology.
> > > > > > > > >
> > > > > > > > > Sure.
> > > > > > > > >
> > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> listed
> > > > two
> > > > > > > > options
> > > > > > > > > without saying which option is chosen.
> > > > > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > > > > >
> > > > > > > > > 5107. Follower to leader transition: It has step 2, but not
> > > step
> > > > 1.
> > > > > > > > > Step-1 is there but it is not explicitly highlighted. It is
> > > > previous
> > > > > > > > > table to step-2.
> > > > > > > > >
> > > > > > > > > 5108. If a consumer fetches from the remote data and the
> remote
> > > > > > storage
> > > > > > > > is
> > > > > > > > > not available, what error code is used in the fetch
> response?
> > > > > > > > >
> > > > > > > > > Good point. We have not yet defined the error for this
> case. We
> > > > need
> > > > > > > > > to define an error message and send the same in fetch
> response.
> > > > > > > > >
> > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the
> first
> > > > message
> > > > > > > > > offset whose timestamp is >= to the given timestamp in the
> > > > request.
> > > > > > That
> > > > > > > > > means it checks in remote log time indexes first, after
> which
> > > > local
> > > > > > log
> > > > > > > > > time indexes are checked." Could you document which method
> in
> > > > RLMM is
> > > > > > > > used
> > > > > > > > > for this?
> > > > > > > > >
> > > > > > > > > Okay.
> > > > > > > > >
> > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> metadata
> > > > of
> > > > > > that
> > > > > > > > > partition with a delete marker and publishes them to RLMM."
> > > This
> > > > > > seems
> > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > >
> > > > > > > > > Will update with KIP-516 related points.
> > > > > > > > >
> > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > earliest
> > > > > > leader
> > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > topicPartition,
> > > > > > > > int
> > > > > > > > > leaderEpoch) and updates that as the log start offset." Do
> we
> > > > need
> > > > > > that
> > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > >
> > > > > > > > > Good point. Right, existing replication protocol takes
> care of
> > > > > > > > > updating the followers’s log start offset received from the
> > > > leader.
> > > > > > > > >
> > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching
> > > from
> > > > > > remote
> > > > > > > > > storage?
> > > > > > > > >
> > > > > > > > > Remote reads may fail within the current default wait
> time, but
> > > > > > > > > subsequent fetches would be able to serve as that data is
> > > stored
> > > > in
> > > > > > > > > the local cache. This cache is currently implemented in
> RSMs.
> > > > But we
> > > > > > > > > plan to pull this into the remote log messaging layer in
> > > future.
> > > > > > > > >
> > > > > > > > > 5113. "Committed offsets can be stored in a local file to
> avoid
> > > > > > reading
> > > > > > > > the
> > > > > > > > > messages again when a broker is restarted." Could you
> describe
> > > > the
> > > > > > format
> > > > > > > > > and the location of the file? Also, could the same message
> be
> > > > > > processed
> > > > > > > > by
> > > > > > > > > RLMM again after broker restart? If so, how do we handle
> that?
> > > > > > > > >
> > > > > > > > > Sure, we will update in the KIP.
> > > > > > > > >
> > > > > > > > > 5114. Message format
> > > > > > > > > 5114.1 There are two records named
> > > RemoteLogSegmentMetadataRecord
> > > > > > with
> > > > > > > > > apiKey 0 and 1.
> > > > > > > > >
> > > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > > >
> > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > > whether
> > > > > > > > endOffset
> > > > > > > > > is inclusive/exclusive?
> > > > > > > > > It is inclusive, will update.
> > > > > > > > >
> > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > > LeaderEpoch
> > > > > > a
> > > > > > > > bit
> > > > > > > > > more? Is that the epoch of the leader when it copies the
> > > segment
> > > > to
> > > > > > > > remote
> > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > >
> > > > > > > > > Right, this is the leader epoch of the broker which copied
> this
> > > > > > > > > segment. This is helpful in reason about which broker
> copied
> > > the
> > > > > > > > > segment to remote storage.
> > > > > > > > >
> > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more?
> Each
> > > > > > record in
> > > > > > > > > Kafka already has a timestamp field. Could we just use
> that?
> > > > > > > > >
> > > > > > > > > This is the  timestamp at which  the respective event
> occurred.
> > > > Added
> > > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > > > > > > > implementation. We thought about that but it looked
> cleaner to
> > > > use at
> > > > > > > > > the message structure level instead of getting that from
> the
> > > > consumer
> > > > > > > > > record and using that to build the respective event.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > >
> > > > > > > > > Right, it looks like config allows only int value >= 14.
> > > > > > > > >
> > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with
> the
> > > log
> > > > > > cleaner
> > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > RemotePartitionRemover.
> > > > > > > > >
> > > > > > > > > I am fine with RemotePartitionRemover or
> > > > RemoteLogDeletionManager(we
> > > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > > >
> > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> processes
> > > it
> > > > if
> > > > > > it is
> > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > >
> > > > > > > > > This is to handle duplicate delete_partition_marked
> events. RLC
> > > > > > > > > internally maintains a state for the delete_partition
> events
> > > and
> > > > if
> > > > > > it
> > > > > > > > > already has an existing event then it ignores if it is
> already
> > > > being
> > > > > > > > > processed.
> > > > > > > > >
> > > > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > > > metadata
> > > > > > > > topic?
> > > > > > > > >
> > > > > > > > > Right, this is in plan but did not mention it in the KIP.
> This
> > > > will
> > > > > > be
> > > > > > > > > useful for debugging purposes too.
> > > > > > > > >
> > > > > > > > > 5118. "Maximum remote log reader thread pool task queue
> size.
> > > If
> > > > the
> > > > > > task
> > > > > > > > > queue is full, broker will stop reading remote log
> segments."
> > > > What
> > > > > > do we
> > > > > > > > > return to the fetch request in this case?
> > > > > > > > >
> > > > > > > > > We return an error response for that partition.
> > > > > > > > >
> > > > > > > > > 5119. It would be useful to list all things not supported
> in
> > > the
> > > > > > first
> > > > > > > > > version in a Future work or Limitations section. For
> example,
> > > > > > compacted
> > > > > > > > > topic, JBOD, changing remote.log.storage.enable from true
> to
> > > > false,
> > > > > > etc.
> > > > > > > > >
> > > > > > > > > We already have a non-goals section which is filled with
> some
> > > of
> > > > > > these
> > > > > > > > > details. Do we need another limitations section?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Satish.
> > > > > > > > >
> > > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > > >
> > > > > > > > > > Hi, Satish,
> > > > > > > > > >
> > > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > > >
> > > > > > > > > > 605.2 "Build the local leader epoch cache by cutting the
> > > leader
> > > > > > epoch
> > > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > > mentioned
> > > > > > an
> > > > > > > > issue
> > > > > > > > > > earlier. Suppose the leader's local start offset is 100.
> The
> > > > > > follower
> > > > > > > > finds
> > > > > > > > > > a remote segment covering offset range [80, 120). The
> > > > producerState
> > > > > > > > with
> > > > > > > > > > this remote segment is up to offset 120. To trim the
> > > > producerState
> > > > > > to
> > > > > > > > > > offset 100 requires more work since one needs to
> download the
> > > > > > previous
> > > > > > > > > > producerState up to offset 80 and then replay the
> messages
> > > > from 80
> > > > > > to
> > > > > > > > 100.
> > > > > > > > > > It seems that it's simpler in this case for the follower
> just
> > > > to
> > > > > > take
> > > > > > > > the
> > > > > > > > > > remote segment as it is and start fetching from offset
> 120.
> > > > > > > > > >
> > > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > > > replicas
> > > > > > for
> > > > > > > > a
> > > > > > > > > > partition, not on the replicas for the
> > > > > > __remote_log_segment_metadata
> > > > > > > > > > partition. It's not clear how the leader of
> > > > > > > > __remote_log_segment_metadata
> > > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > > >
> > > > > > > > > > 5100. KIP-516 has been accepted and is being implemented
> now.
> > > > > > Could you
> > > > > > > > > > update the KIP based on topicID?
> > > > > > > > > >
> > > > > > > > > > 5101. RLMM: It would be useful to clarify how the
> following
> > > two
> > > > > > APIs
> > > > > > > > are
> > > > > > > > > > used. According to the wiki, the former is used for topic
> > > > deletion
> > > > > > and
> > > > > > > > the
> > > > > > > > > > latter is used for retention. It seems that retention
> should
> > > > use
> > > > > > the
> > > > > > > > former
> > > > > > > > > > since remote segments without a matching epoch in the
> leader
> > > > > > > > (potentially
> > > > > > > > > > due to unclean leader election) also need to be garbage
> > > > collected.
> > > > > > The
> > > > > > > > > > latter seems to be used for the new leader to determine
> the
> > > > last
> > > > > > tiered
> > > > > > > > > > segment.
> > > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > > >
> > > > > > > > > > 5102. RSM:
> > > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems
> that
> > > > they
> > > > > > can
> > > > > > > > > > use RemoteLogSegmentId instead of
> RemoteLogSegmentMetadata.
> > > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long
> instead
> > > of
> > > > > > Long?
> > > > > > > > > > 5102.3 Why only some of the methods have default
> > > > implementation and
> > > > > > > > others
> > > > > > > > > > don't?
> > > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a
> file
> > > > > > since it
> > > > > > > > will
> > > > > > > > > > be generated in memory.
> > > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs
> both
> > > > > > > > baseOffset and
> > > > > > > > > > startOffset. For example, deleteRecords() could move the
> > > > > > startOffset
> > > > > > > > to the
> > > > > > > > > > middle of a segment. If we copy the full segment to
> remote
> > > > > > storage, the
> > > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > > and LogSegmentData?
> > > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > > RemoteLogSegmentMetadata
> > > > > > > > is
> > > > > > > > > > inclusive/exclusive?
> > > > > > > > > >
> > > > > > > > > > 5103. configs:
> > > > > > > > > > 5103.1 Could we define the default value of non-required
> > > > configs
> > > > > > (e.g
> > > > > > > > the
> > > > > > > > > > size of new thread pools)?
> > > > > > > > > > 5103.2 It seems that local.log.retention.ms should
> default
> > > to
> > > > > > > > retention.ms,
> > > > > > > > > > instead of remote.log.retention.minutes. Similarly, it
> seems
> > > > > > > > > > that local.log.retention.bytes should default to
> > > segment.bytes.
> > > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The
> description
> > > > says
> > > > > > "used
> > > > > > > > in
> > > > > > > > > > scheduling tasks to copy segments, fetch remote log
> indexes
> > > and
> > > > > > clean
> > > > > > > > up
> > > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > > config remote.log.reader.threads for fetching remote
> data.
> > > It's
> > > > > > weird
> > > > > > > > to
> > > > > > > > > > fetch remote index and log in different thread pools
> since
> > > > both are
> > > > > > > > used
> > > > > > > > > > for serving fetch requests.
> > > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> > > > amount of
> > > > > > > > time to
> > > > > > > > > > back off when there is no work to do? If so, perhaps it
> can
> > > be
> > > > > > renamed
> > > > > > > > as
> > > > > > > > > > backoff.ms.
> > > > > > > > > > 5103.5 Are rlm_process_interval_ms and
> rlm_retry_interval_ms
> > > > > > configs?
> > > > > > > > If
> > > > > > > > > > so, they need to be listed in this section.
> > > > > > > > > >
> > > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the
> > > index
> > > > > > files
> > > > > > > > of
> > > > > > > > > > remote log segments to avoid multiple index fetches from
> the
> > > > remote
> > > > > > > > > > storage." Is the RLM in memory or on disk? If on disk,
> where
> > > > is it
> > > > > > > > stored?
> > > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > > >
> > > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest
> Local
> > > > > > Offset in
> > > > > > > > > > different places. It would be useful to standardize the
> > > > > > terminology.
> > > > > > > > > >
> > > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> > > > listed two
> > > > > > > > options
> > > > > > > > > > without saying which option is chosen.
> > > > > > > > > >
> > > > > > > > > > 5107. Follower to leader transition: It has step 2, but
> not
> > > > step 1.
> > > > > > > > > >
> > > > > > > > > > 5108. If a consumer fetches from the remote data and the
> > > remote
> > > > > > > > storage is
> > > > > > > > > > not available, what error code is used in the fetch
> response?
> > > > > > > > > >
> > > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the
> first
> > > > > > message
> > > > > > > > > > offset whose timestamp is >= to the given timestamp in
> the
> > > > request.
> > > > > > > > That
> > > > > > > > > > means it checks in remote log time indexes first, after
> which
> > > > > > local log
> > > > > > > > > > time indexes are checked." Could you document which
> method in
> > > > RLMM
> > > > > > is
> > > > > > > > used
> > > > > > > > > > for this?
> > > > > > > > > >
> > > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > > > metadata of
> > > > > > that
> > > > > > > > > > partition with a delete marker and publishes them to
> RLMM."
> > > > This
> > > > > > seems
> > > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > > >
> > > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > > earliest
> > > > > > leader
> > > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > > topicPartition,
> > > > > > > > int
> > > > > > > > > > leaderEpoch) and updates that as the log start offset."
> Do we
> > > > need
> > > > > > that
> > > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > > >
> > > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for
> fetching
> > > > from
> > > > > > remote
> > > > > > > > > > storage?
> > > > > > > > > >
> > > > > > > > > > 5113. "Committed offsets can be stored in a local file to
> > > avoid
> > > > > > > > reading the
> > > > > > > > > > messages again when a broker is restarted." Could you
> > > describe
> > > > the
> > > > > > > > format
> > > > > > > > > > and the location of the file? Also, could the same
> message be
> > > > > > > > processed by
> > > > > > > > > > RLMM again after broker restart? If so, how do we handle
> > > that?
> > > > > > > > > >
> > > > > > > > > > 5114. Message format
> > > > > > > > > > 5114.1 There are two records named
> > > > RemoteLogSegmentMetadataRecord
> > > > > > with
> > > > > > > > > > apiKey 0 and 1.
> > > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > > > whether
> > > > > > > > endOffset
> > > > > > > > > > is inclusive/exclusive?
> > > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > > > > LeaderEpoch a
> > > > > > > > bit
> > > > > > > > > > more? Is that the epoch of the leader when it copies the
> > > > segment to
> > > > > > > > remote
> > > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more?
> > > Each
> > > > > > record
> > > > > > > > in
> > > > > > > > > > Kafka already has a timestamp field. Could we just use
> that?
> > > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > > >
> > > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with
> the
> > > > log
> > > > > > > > cleaner
> > > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > > RemotePartitionRemover.
> > > > > > > > > >
> > > > > > > > > > 5116. "RLC receives the delete_partition_marked and
> processes
> > > > it
> > > > > > if it
> > > > > > > > is
> > > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > > >
> > > > > > > > > > 5117. Should we add a new MessageFormatter to read the
> tier
> > > > > > metadata
> > > > > > > > topic?
> > > > > > > > > >
> > > > > > > > > > 5118. "Maximum remote log reader thread pool task queue
> size.
> > > > If
> > > > > > the
> > > > > > > > task
> > > > > > > > > > queue is full, broker will stop reading remote log
> segments."
> > > > What
> > > > > > do
> > > > > > > > we
> > > > > > > > > > return to the fetch request in this case?
> > > > > > > > > >
> > > > > > > > > > 5119. It would be useful to list all things not
> supported in
> > > > the
> > > > > > first
> > > > > > > > > > version in a Future work or Limitations section. For
> example,
> > > > > > compacted
> > > > > > > > > > topic, JBOD, changing remote.log.storage.enable from
> true to
> > > > false,
> > > > > > > > etc.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > > (
> > > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> > >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Jun,
Thanks for your comments. Please find the inline replies below.

6022. For packages used for server plugins, the convention is to
use org.apache.kafka.server. See java-based Authorizer as an example.

Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to
‘org.apache.kafka.server.log.remote.storage’.  Updated in the KIP.

9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat
file captures the state of the remote segments. After a partition is
deleted, it seems that we just need to remove the partitions's remote
segments from the flat file.

DeletePartitionState might not yet have been processed by RPM and not
completed. We  will not have  that in flat  file format  once it
reaches DELETE_PARTITION_FINISHED state.

9101. Upgrade: It will be useful to allow direct upgrade from an old
version. It seems that's doable. One can just do the normal upgrade first
and wait enough time (for producer snapshots to be built), and then enable
remote storage.

Upgrade notes updates in the KIP.

9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
tracking the remote segments when RLMM.onPartitionLeadershipChanges() is
called with the broker being the leader for __remote_log_metadata
partition? If so, could we document it?

The current plan is to have that as part of RLMM and RPM uses that to
get the remote segments list. I will add this detail in the respective
sections.

Satish.

On Wed, 16 Dec 2020 at 23:55, Jun Rao <ju...@confluent.io> wrote:
>
> Hi, Satish,
>
> Thanks for the reply. A few more followup comments.
>
> 6022. For packages used for server plugins, the convention is to
> use org.apache.kafka.server. See java-based Authorizer as an example.
>
> 9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat
> file captures the state of the remote segments. After a partition is
> deleted, it seems that we just need to remove the partitions's remote
> segments from the flat file.
>
> 9101. Upgrade: It will be useful to allow direct upgrade from an old
> version. It seems that's doable. One can just do the normal upgrade first
> and wait enough time (for producer snapshots to be built), and then enable
> remote storage.
>
> 9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
> tracking the remote segments when RLMM.onPartitionLeadershipChanges() is
> called with the broker being the leader for __remote_log_metadata
> partition? If so, could we document it?
>
> Jun
>
> On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <kp...@confluent.io>
> wrote:
>
> > Hi Satish,
> >
> > Thanks for the updates! A few more comments below.
> >
> > 9001. Under the "Upgrade" section, there is a line mentioning: "Upgrade the
> > existing Kafka cluster to 2.7 version and allow this to run for the log
> > retention of user topics that you want to enable tiered storage. This will
> > allow all the topics to have the producer snapshots generated for each log
> > segment." -- Which associated change in AK were you referring to here? Is
> > it: https://github.com/apache/kafka/pull/7929 ? It seems like I don't see
> > it in the 2.7 release branch yet, here is the link:
> > https://github.com/apache/kafka/commits/2.7.
> >
> > 9002. Under the "Upgrade" section, the configuration mentioned is
> > 'remote.log.storage.system.enable'. However, under "Public Interfaces"
> > section the corresponding configuration is 'remote.storage.system.enable'.
> > Could we use the same one in both, maybe
> > 'remote.log.storage.system.enable'?
> >
> > 9003. Under "Per Topic Configuration", the KIP recommends setting
> > 'remote.log.storage.enable' to true at a per-topic level. It will be useful
> > to add a line that if the user wants to enable it for all topics, then they
> > should be able to set the cluster-wide default to true. Also, it will be
> > useful to mention that the KIP currently does not support setting it to
> > false (after it is set to true), and add that to the future work section.
> >
> > 9004. Under "Committed offsets file format", the sample provided shows
> > partition number and offset. Is the topic name required for identifying
> > which topic the partitions belong to?
> >
> > 9005. Under "Internal flat-file store format of remote log metadata", it
> > seems useful to specify both topic name and topic ID for debugging
> > purposes.
> >
> > 9006. Under "Internal flat-file store format of remote log metadata", the
> > description of "metadata-topic-offset" currently says "offset of the remote
> > log metadata topic from which this topic partition's remote log metadata is
> > fetched." Just for the wording, perhaps you meant to refer to the offset
> > upto which the file has been committed? i.e. "offset of the remote log
> > metadata topic upto which this topic partition's remote log metadata has
> > been committed into this file."
> >
> > 9007. Under "Internal flat-file store format of remote log metadata", the
> > schema of the payload (i.e. beyond the header) seems to contain the events
> > from the metadata topic. It seems useful to instead persist the
> > representation of the materialized state of the events, so that for the
> > same segment only the latest state is stored. Besides reducing storage
> > footprint, this also is likely to relate directly with the in-memory
> > representation of the RLMM cache (which probably is some kind of a Map with
> > key being segment ID and value being the segment state), so recovery from
> > disk will be straightforward.
> >
> > 9008. Under "Topic deletion lifecycle", step (1), it will be useful to
> > mention when in the deletion flow does the controller publish the
> > delete_partition_marked event to say that the partition is marked for
> > deletion?
> >
> > 9009. There are ~4 TODOs in the KIP. Could you please address these or
> > remove them?
> >
> > 9010. There is a reference to a Google doc on the KIP which was used
> > earlier for discussions. Please could you remove the reference, since the
> > KIP is the source of the truth?
> >
> > 9011. This feedback is from an earlier comment. 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. What do you think?
> >
> >
> > Cheers,
> > Kowshik
> >
> >
> > On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <sa...@gmail.com>
> > wrote:
> >
> > > Hi Jun,
> > > Thanks for your comments. Please go through the inline replies.
> > >
> > >
> > > 5102.2: It seems that both positions can just be int. Another option is
> > to
> > > have two methods. Would it be clearer?
> > >
> > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > remoteLogSegmentMetadata,  int startPosition)
> > throwsRemoteStorageException;
> > >
> > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > remoteLogSegmentMetadata, int startPosition, int endPosition) throws
> > > RemoteStorageException;
> > >
> > > That makes sense to me, updated the KIP.
> > >
> > > 6003: Could you also update the javadoc for the return value?
> > >
> > > Updated.
> > >
> > > 6020: local.log.retention.bytes: Should it default to log.retention.bytes
> > > to be consistent with local.log.retention.ms?
> > >
> > > Yes, it can be defaulted to log.retention.bytes.
> > >
> > > 6021: Could you define TopicIdPartition?
> > >
> > > Added TopicIdPartition in the KIP.
> > >
> > > 6022: For all public facing classes, could you specify the package name?
> > >
> > > Updated.
> > >
> > >
> > > Thanks,
> > > Satish.
> > >
> > > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > Hi, Satish,
> > > >
> > > > Thanks for the reply. A few more comments below.
> > > >
> > > > 5102.2: It seems that both positions can just be int. Another option is
> > > to
> > > > have two methods. Would it be clearer?
> > > >
> > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > remoteLogSegmentMetadata,
> > > >                                     int startPosition) throws
> > > > RemoteStorageException;
> > > >
> > > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > > remoteLogSegmentMetadata,
> > > >                                     int startPosition, int endPosition)
> > > > throws RemoteStorageException;
> > > >
> > > > 6003: Could you also update the javadoc for the return value?
> > > >
> > > > 6010: What kind of tiering throughput have you seen with 5 threads?
> > > >
> > > > 6020: local.log.retention.bytes: Should it default to
> > log.retention.bytes
> > > > to be consistent with local.log.retention.ms?
> > > >
> > > > 6021: Could you define TopicIdPartition?
> > > >
> > > > 6022: For all public facing classes, could you specify the package
> > name?
> > > >
> > > > It seems that you already added the topicId support. Two other
> > remaining
> > > > items are (a) the format of local tier metadata storage and (b)
> > upgrade.
> > > >
> > > > Jun
> > > >
> > > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> > satish.duggana@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Jun,
> > > > > Thanks for your comments. Please find the inline replies below.
> > > > >
> > > > > >605.2 It's rare for the follower to need the remote data. So, the
> > > current
> > > > > approach is fine too. Could you document the process of rebuilding
> > the
> > > > > producer state since we can't simply trim the producerState to an
> > > offset in
> > > > > the middle of a segment.
> > > > >
> > > > > Will clarify in the KIP.
> > > > >
> > > > > >5102.2 Would it be clearer to make startPosiont long and endPosition
> > > of
> > > > > Optional<Long>?
> > > > >
> > > > > We will have arg checks with respective validation. It is not a good
> > > > > practice to have arguments with optional as mentioned here.
> > > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > > >
> > > > >
> > > > > >5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > > > ByteBuffer.
> > > > >
> > > > > Updated.
> > > > >
> > > > > >5102.7 Could you define all public methods for LogSegmentData?
> > > > >
> > > > > Updated.
> > > > >
> > > > > >5103.5 Could you change the reference to rlm_process_interval_ms and
> > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > interval
> > > > > config seems still missing. It would be useful to support exponential
> > > > > backoff with the retry interval config.
> > > > >
> > > > > Good point. We wanted the retry with truncated exponential backoff,
> > > > > updated the KIP.
> > > > >
> > > > > >5111. "RLM follower fetches the earliest offset for the earliest
> > > leader
> > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > topicPartition,
> > > int
> > > > > leaderEpoch) and updates that as the log start offset." This text is
> > > still
> > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > >
> > > > > Updated.
> > > > >
> > > > > >5115. There are still references to "remote log cleaners".
> > > > >
> > > > > Updated.
> > > > >
> > > > > >6000. Since we are returning new error codes, we need to bump up the
> > > > > protocol version for Fetch request. Also, it will be useful to
> > > document all
> > > > > new error codes and whether they are retriable or not.
> > > > >
> > > > > Sure, we will add that in the KIP.
> > > > >
> > > > > >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > leaderEpoch
> > > > > is int32 instead of long.
> > > > >
> > > > > Updated.
> > > > >
> > > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > > > RemoteLogSegmentMetadata.state()?
> > > > >
> > > > > No, it is fixed.
> > > > >
> > > > > >6003. RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata(TopicPartition
> > > > > topicPartition, long offset, int epochForOffset): Should this return
> > > > > Optional<RemoteLogSegmentMetadata>?
> > > > >
> > > > > That makes sense, updated.
> > > > >
> > > > > >6005. RemoteLogState: It seems it's better to split it between
> > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the
> > > states
> > > > > are never shared between the two use cases.
> > > > >
> > > > > Agree with that, updated.
> > > > >
> > > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However,
> > > is it
> > > > > ture that other than the metadata topic, RLMM just needs to know
> > > whether
> > > > > there is a replica assigned to this broker and doesn't need to know
> > > whether
> > > > > the replica is the leader or the follower?
> > > > >
> > > > > That may be true. If the implementation does not need that, it can
> > > > > ignore the information in the callback.
> > > > >
> > > > > >6007: "Handle expired remote segments (leader and follower)": Why is
> > > this
> > > > > needed in both the leader and the follower?
> > > > >
> > > > > Updated.
> > > > >
> > > > > >6008.       "name": "SegmentSizeInBytes",
> > > > >                 "type": "int64",
> > > > > The segment size can just be int32.
> > > > >
> > > > > Updated.
> > > > >
> > > > > >6009. For the record format in the log, it seems that we need to add
> > > > > record
> > > > > type and record version before the serialized bytes. We can follow
> > the
> > > > > convention used in
> > > > >
> > > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > >
> > > > > Yes, KIP already mentions that these are serialized before the
> > payload
> > > > > as below. We will mention explicitly that these two are written
> > before
> > > > > the data is written.
> > > > >
> > > > > RLMM instance on broker publishes the message to the topic with key
> > as
> > > > > null and value with the below format.
> > > > >
> > > > > type      : unsigned var int, represents the value type. This value
> > is
> > > > > 'apikey' as mentioned in the schema.
> > > > > version : unsigned var int, the 'version' number of the type as
> > > > > mentioned in the schema.
> > > > > data      : record payload in kafka protocol message format.
> > > > >
> > > > >
> > > > > >6010. remote.log.manager.thread.pool.size: The default value is 10.
> > > This
> > > > > might be too high when enabling the tiered feature for the first
> > time.
> > > > > Since there are lots of segments that need to be tiered initially, a
> > > large
> > > > > number of threads could overwhelm the broker.
> > > > >
> > > > > Is the default value 5 reasonable?
> > > > >
> > > > > 6011. "The number of milli seconds to keep the local log segment
> > > before it
> > > > > gets deleted. If not set, the value in `log.retention.minutes` is
> > > used. If
> > > > > set to -1, no time limit is applied." We should use log.retention.ms
> > > > > instead of log.retention.minutes.
> > > > > Nice typo catch. Updated the KIP.
> > > > >
> > > > > Thanks,
> > > > > Satish.
> > > > >
> > > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > Hi, Satish,
> > > > > >
> > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > >
> > > > > > 605.2 It's rare for the follower to need the remote data. So, the
> > > current
> > > > > > approach is fine too. Could you document the process of rebuilding
> > > the
> > > > > > producer state since we can't simply trim the producerState to an
> > > offset
> > > > > in
> > > > > > the middle of a segment.
> > > > > >
> > > > > > 5102.2 Would it be clearer to make startPosiont long and
> > endPosition
> > > of
> > > > > > Optional<Long>?
> > > > > >
> > > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > > > > ByteBuffer.
> > > > > >
> > > > > > 5102.7 Could you define all public methods for LogSegmentData?
> > > > > >
> > > > > > 5103.5 Could you change the reference to rlm_process_interval_ms
> > and
> > > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > > interval
> > > > > > config seems still missing. It would be useful to support
> > exponential
> > > > > > backoff with the retry interval config.
> > > > > >
> > > > > > 5111. "RLM follower fetches the earliest offset for the earliest
> > > leader
> > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > topicPartition,
> > > > > int
> > > > > > leaderEpoch) and updates that as the log start offset." This text
> > is
> > > > > still
> > > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > > >
> > > > > > 5115. There are still references to "remote log cleaners".
> > > > > >
> > > > > > 6000. Since we are returning new error codes, we need to bump up
> > the
> > > > > > protocol version for Fetch request. Also, it will be useful to
> > > document
> > > > > all
> > > > > > new error codes and whether they are retriable or not.
> > > > > >
> > > > > > 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > > > leaderEpoch
> > > > > > is int32 instead of long.
> > > > > >
> > > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > > > > RemoteLogSegmentMetadata.state()?
> > > > > >
> > > > > > 6003. RemoteLogSegmentMetadata
> > > remoteLogSegmentMetadata(TopicPartition
> > > > > > topicPartition, long offset, int epochForOffset): Should this
> > return
> > > > > > Optional<RemoteLogSegmentMetadata>?
> > > > > >
> > > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to pick a
> > > more
> > > > > > indicative name so that people understand what epoch this is.
> > > > > >
> > > > > > 6005. RemoteLogState: It seems it's better to split it between
> > > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the
> > > states
> > > > > > are never shared between the two use cases.
> > > > > >
> > > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However,
> > > is it
> > > > > > ture that other than the metadata topic, RLMM just needs to know
> > > whether
> > > > > > there is a replica assigned to this broker and doesn't need to know
> > > > > whether
> > > > > > the replica is the leader or the follower?
> > > > > >
> > > > > > 6007: "Handle expired remote segments (leader and follower)": Why
> > is
> > > this
> > > > > > needed in both the leader and the follower?
> > > > > >
> > > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > > >                 "type": "int64",
> > > > > > The segment size can just be int32.
> > > > > >
> > > > > > 6009. For the record format in the log, it seems that we need to
> > add
> > > > > record
> > > > > > type and record version before the serialized bytes. We can follow
> > > the
> > > > > > convention used in
> > > > > >
> > > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > > .
> > > > > >
> > > > > > 6010. remote.log.manager.thread.pool.size: The default value is 10.
> > > This
> > > > > > might be too high when enabling the tiered feature for the first
> > > time.
> > > > > > Since there are lots of segments that need to be tiered initially,
> > a
> > > > > large
> > > > > > number of threads could overwhelm the broker.
> > > > > >
> > > > > > 6011. "The number of milli seconds to keep the local log segment
> > > before
> > > > > it
> > > > > > gets deleted. If not set, the value in `log.retention.minutes` is
> > > used.
> > > > > If
> > > > > > set to -1, no time limit is applied." We should use
> > log.retention.ms
> > > > > > instead of log.retention.minutes.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi,
> > > > > > > We updated the KIP with the points mentioned in the earlier mail
> > > > > > > except for KIP-516 related changes. You can go through them and
> > > let us
> > > > > > > know if you have any comments. We will update the KIP with the
> > > > > > > remaining todo items and KIP-516 related changes by end of this
> > > > > > > week(5th Dec).
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > > satish.duggana@gmail.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > Hi Jun,
> > > > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > > > >
> > > > > > > > 605.2 "Build the local leader epoch cache by cutting the leader
> > > epoch
> > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > mentioned an
> > > > > > > issue
> > > > > > > > earlier. Suppose the leader's local start offset is 100. The
> > > follower
> > > > > > > finds
> > > > > > > > a remote segment covering offset range [80, 120). The
> > > producerState
> > > > > with
> > > > > > > > this remote segment is up to offset 120. To trim the
> > > producerState to
> > > > > > > > offset 100 requires more work since one needs to download the
> > > > > previous
> > > > > > > > producerState up to offset 80 and then replay the messages from
> > > 80 to
> > > > > > > 100.
> > > > > > > > It seems that it's simpler in this case for the follower just
> > to
> > > > > take the
> > > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > > >
> > > > > > > > We chose that approach to avoid any edge cases here. It may be
> > > > > > > > possible that the remote log segment that is received may not
> > > have
> > > > > the
> > > > > > > > same leader epoch sequence from 100-120 as it contains on the
> > > > > > > > leader(this can happen due to unclean leader). It is safe to
> > > start
> > > > > > > > from what the leader returns here.Another way is to find the
> > > remote
> > > > > > > > log segment
> > > > > > > >
> > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > > replicas
> > > > > for a
> > > > > > > > partition, not on the replicas for the
> > > __remote_log_segment_metadata
> > > > > > > > partition. It's not clear how the leader of
> > > > > __remote_log_segment_metadata
> > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > >
> > > > > > > > RLMM will always receive the callback for the remote log
> > metadata
> > > > > > > > topic partitions hosted on the local broker and these will be
> > > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > > >
> > > > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> > > Could
> > > > > you
> > > > > > > > update the KIP based on topicID?
> > > > > > > >
> > > > > > > > We mentioned KIP-516 and how it helps. We will update this KIP
> > > with
> > > > > > > > all the changes it brings with KIP-516.
> > > > > > > >
> > > > > > > > 5101. RLMM: It would be useful to clarify how the following two
> > > APIs
> > > > > are
> > > > > > > > used. According to the wiki, the former is used for topic
> > > deletion
> > > > > and
> > > > > > > the
> > > > > > > > latter is used for retention. It seems that retention should
> > use
> > > the
> > > > > > > former
> > > > > > > > since remote segments without a matching epoch in the leader
> > > > > (potentially
> > > > > > > > due to unclean leader election) also need to be garbage
> > > collected.
> > > > > The
> > > > > > > > latter seems to be used for the new leader to determine the
> > last
> > > > > tiered
> > > > > > > > segment.
> > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > >
> > > > > > > > Right,.that is what we are currently doing. We will update the
> > > > > > > > javadocs and wiki with that. Earlier, we did not want to remove
> > > the
> > > > > > > > segments which are not matched with leader epochs from the
> > ladder
> > > > > > > > partition as they may be used later by a replica which can
> > > become a
> > > > > > > > leader (unclean leader election) and refer those segments. But
> > > that
> > > > > > > > may leak these segments in remote storage until the topic
> > > lifetime.
> > > > > We
> > > > > > > > decided to cleanup the segments with the oldest incase of size
> > > based
> > > > > > > > retention also.
> > > > > > > >
> > > > > > > > 5102. RSM:
> > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that
> > > they can
> > > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > > >
> > > > > > > > It will be useful to have metadata for RSM to fetch log
> > segment.
> > > It
> > > > > > > > may create location/path using id with other metadata too.
> > > > > > > >
> > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of
> > > Long?
> > > > > > > >
> > > > > > > > Wanted to keep endPosition as optional to read till the end of
> > > the
> > > > > > > > segment and avoid sentinels.
> > > > > > > >
> > > > > > > > 5102.3 Why only some of the methods have default implementation
> > > and
> > > > > > > others
> > > > > > > > Don't?
> > > > > > > >
> > > > > > > > Actually,  RSM will not have any default implementations.
> > Those 3
> > > > > > > > methods were made default earlier for tests etc. Updated the
> > > wiki.
> > > > > > > >
> > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > and DeletePartitionUpdate?
> > > > > > > >
> > > > > > > > Sure, they will be added.
> > > > > > > >
> > > > > > > >
> > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> > > since
> > > > > it
> > > > > > > will
> > > > > > > > be generated in memory.
> > > > > > > >
> > > > > > > > Right, this is in plan.
> > > > > > > >
> > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > > > baseOffset
> > > > > > > and
> > > > > > > > startOffset. For example, deleteRecords() could move the
> > > startOffset
> > > > > to
> > > > > > > the
> > > > > > > > middle of a segment. If we copy the full segment to remote
> > > storage,
> > > > > the
> > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > >
> > > > > > > > Good point. startOffset is baseOffset by default, if not set
> > > > > explicitly.
> > > > > > > >
> > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > RemoteLogSegmentMetadata
> > > > > > > > and LogSegmentData?
> > > > > > > >
> > > > > > > > Sure, updated the wiki.
> > > > > > > >
> > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > RemoteLogSegmentMetadata is
> > > > > > > > inclusive/exclusive?
> > > > > > > >
> > > > > > > > It is inclusive, will update.
> > > > > > > >
> > > > > > > > 5103. configs:
> > > > > > > > 5103.1 Could we define the default value of non-required
> > configs
> > > > > (e.g the
> > > > > > > > size of new thread pools)?
> > > > > > > >
> > > > > > > > Sure, that makes sense.
> > > > > > > >
> > > > > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > > > > > retention.ms,
> > > > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > > > > >
> > > > > > > > Right, we do not have  remote.log.retention as we discussed
> > > earlier.
> > > > > > > > Thanks for catching the typo.
> > > > > > > >
> > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description
> > says
> > > > > "used in
> > > > > > > > scheduling tasks to copy segments, fetch remote log indexes and
> > > > > clean up
> > > > > > > > remote log segments". However, there is a separate
> > > > > > > > config remote.log.reader.threads for fetching remote data. It's
> > > > > weird to
> > > > > > > > fetch remote index and log in different thread pools since both
> > > are
> > > > > used
> > > > > > > > for serving fetch requests.
> > > > > > > >
> > > > > > > > Right, remote.log.manager.thread.pool is mainly used for
> > > copy/cleanup
> > > > > > > > activities. Fetch path always goes through
> > > remote.log.reader.threads.
> > > > > > > >
> > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount
> > > of
> > > > > time
> > > > > > > to
> > > > > > > > back off when there is no work to do? If so, perhaps it can be
> > > > > renamed as
> > > > > > > > backoff.ms.
> > > > > > > >
> > > > > > > > This is the delay interval for each iteration. It may be
> > renamed
> > > to
> > > > > > > > remote.log.manager.task.delay.ms
> > > > > > > >
> > > > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > > > > configs? If
> > > > > > > > so, they need to be listed in this section.
> > > > > > > >
> > > > > > > > remote.log.manager.task.interval.ms is the process internal,
> > > retry
> > > > > > > > interval is missing in the configs, which will be updated in
> > the
> > > KIP.
> > > > > > > >
> > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> > > > > files of
> > > > > > > > remote log segments to avoid multiple index fetches from the
> > > remote
> > > > > > > > storage." Is the RLM in memory or on disk? If on disk, where is
> > > it
> > > > > > > stored?
> > > > > > > > Do we need a configuration to bound the size?
> > > > > > > >
> > > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > > `remote-log-index-cache` under log dir. We plan to have a
> > config
> > > for
> > > > > > > > that instead of default. We will have a configuration for that.
> > > > > > > >
> > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> > > Offset
> > > > > in
> > > > > > > > different places. It would be useful to standardize the
> > > terminology.
> > > > > > > >
> > > > > > > > Sure.
> > > > > > > >
> > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It listed
> > > two
> > > > > > > options
> > > > > > > > without saying which option is chosen.
> > > > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > > > >
> > > > > > > > 5107. Follower to leader transition: It has step 2, but not
> > step
> > > 1.
> > > > > > > > Step-1 is there but it is not explicitly highlighted. It is
> > > previous
> > > > > > > > table to step-2.
> > > > > > > >
> > > > > > > > 5108. If a consumer fetches from the remote data and the remote
> > > > > storage
> > > > > > > is
> > > > > > > > not available, what error code is used in the fetch response?
> > > > > > > >
> > > > > > > > Good point. We have not yet defined the error for this case. We
> > > need
> > > > > > > > to define an error message and send the same in fetch response.
> > > > > > > >
> > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> > > message
> > > > > > > > offset whose timestamp is >= to the given timestamp in the
> > > request.
> > > > > That
> > > > > > > > means it checks in remote log time indexes first, after which
> > > local
> > > > > log
> > > > > > > > time indexes are checked." Could you document which method in
> > > RLMM is
> > > > > > > used
> > > > > > > > for this?
> > > > > > > >
> > > > > > > > Okay.
> > > > > > > >
> > > > > > > > 5110. Stopreplica: "it sets all the remote log segment metadata
> > > of
> > > > > that
> > > > > > > > partition with a delete marker and publishes them to RLMM."
> > This
> > > > > seems
> > > > > > > > outdated given the new topic deletion logic.
> > > > > > > >
> > > > > > > > Will update with KIP-516 related points.
> > > > > > > >
> > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > earliest
> > > > > leader
> > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > topicPartition,
> > > > > > > int
> > > > > > > > leaderEpoch) and updates that as the log start offset." Do we
> > > need
> > > > > that
> > > > > > > > since replication propagates logStartOffset already?
> > > > > > > >
> > > > > > > > Good point. Right, existing replication protocol takes care of
> > > > > > > > updating the followers’s log start offset received from the
> > > leader.
> > > > > > > >
> > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching
> > from
> > > > > remote
> > > > > > > > storage?
> > > > > > > >
> > > > > > > > Remote reads may fail within the current default wait time, but
> > > > > > > > subsequent fetches would be able to serve as that data is
> > stored
> > > in
> > > > > > > > the local cache. This cache is currently implemented in RSMs.
> > > But we
> > > > > > > > plan to pull this into the remote log messaging layer in
> > future.
> > > > > > > >
> > > > > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > > > > reading
> > > > > > > the
> > > > > > > > messages again when a broker is restarted." Could you describe
> > > the
> > > > > format
> > > > > > > > and the location of the file? Also, could the same message be
> > > > > processed
> > > > > > > by
> > > > > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > > > > >
> > > > > > > > Sure, we will update in the KIP.
> > > > > > > >
> > > > > > > > 5114. Message format
> > > > > > > > 5114.1 There are two records named
> > RemoteLogSegmentMetadataRecord
> > > > > with
> > > > > > > > apiKey 0 and 1.
> > > > > > > >
> > > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > > >
> > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > whether
> > > > > > > endOffset
> > > > > > > > is inclusive/exclusive?
> > > > > > > > It is inclusive, will update.
> > > > > > > >
> > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > LeaderEpoch
> > > > > a
> > > > > > > bit
> > > > > > > > more? Is that the epoch of the leader when it copies the
> > segment
> > > to
> > > > > > > remote
> > > > > > > > storage? Also, how will this field be used?
> > > > > > > >
> > > > > > > > Right, this is the leader epoch of the broker which copied this
> > > > > > > > segment. This is helpful in reason about which broker copied
> > the
> > > > > > > > segment to remote storage.
> > > > > > > >
> > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> > > > > record in
> > > > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > > > >
> > > > > > > > This is the  timestamp at which  the respective event occurred.
> > > Added
> > > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > > > > > > implementation. We thought about that but it looked cleaner to
> > > use at
> > > > > > > > the message structure level instead of getting that from the
> > > consumer
> > > > > > > > record and using that to build the respective event.
> > > > > > > >
> > > > > > > >
> > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > >
> > > > > > > > Right, it looks like config allows only int value >= 14.
> > > > > > > >
> > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the
> > log
> > > > > cleaner
> > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > RemotePartitionRemover.
> > > > > > > >
> > > > > > > > I am fine with RemotePartitionRemover or
> > > RemoteLogDeletionManager(we
> > > > > > > > have other manager classes like RLM, RLMM).
> > > > > > > >
> > > > > > > > 5116. "RLC receives the delete_partition_marked and processes
> > it
> > > if
> > > > > it is
> > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > >
> > > > > > > > This is to handle duplicate delete_partition_marked events. RLC
> > > > > > > > internally maintains a state for the delete_partition events
> > and
> > > if
> > > > > it
> > > > > > > > already has an existing event then it ignores if it is already
> > > being
> > > > > > > > processed.
> > > > > > > >
> > > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > > metadata
> > > > > > > topic?
> > > > > > > >
> > > > > > > > Right, this is in plan but did not mention it in the KIP. This
> > > will
> > > > > be
> > > > > > > > useful for debugging purposes too.
> > > > > > > >
> > > > > > > > 5118. "Maximum remote log reader thread pool task queue size.
> > If
> > > the
> > > > > task
> > > > > > > > queue is full, broker will stop reading remote log segments."
> > > What
> > > > > do we
> > > > > > > > return to the fetch request in this case?
> > > > > > > >
> > > > > > > > We return an error response for that partition.
> > > > > > > >
> > > > > > > > 5119. It would be useful to list all things not supported in
> > the
> > > > > first
> > > > > > > > version in a Future work or Limitations section. For example,
> > > > > compacted
> > > > > > > > topic, JBOD, changing remote.log.storage.enable from true to
> > > false,
> > > > > etc.
> > > > > > > >
> > > > > > > > We already have a non-goals section which is filled with some
> > of
> > > > > these
> > > > > > > > details. Do we need another limitations section?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Satish.
> > > > > > > >
> > > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > > >
> > > > > > > > > Hi, Satish,
> > > > > > > > >
> > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > >
> > > > > > > > > 605.2 "Build the local leader epoch cache by cutting the
> > leader
> > > > > epoch
> > > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > > mentioned
> > > > > an
> > > > > > > issue
> > > > > > > > > earlier. Suppose the leader's local start offset is 100. The
> > > > > follower
> > > > > > > finds
> > > > > > > > > a remote segment covering offset range [80, 120). The
> > > producerState
> > > > > > > with
> > > > > > > > > this remote segment is up to offset 120. To trim the
> > > producerState
> > > > > to
> > > > > > > > > offset 100 requires more work since one needs to download the
> > > > > previous
> > > > > > > > > producerState up to offset 80 and then replay the messages
> > > from 80
> > > > > to
> > > > > > > 100.
> > > > > > > > > It seems that it's simpler in this case for the follower just
> > > to
> > > > > take
> > > > > > > the
> > > > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > > > >
> > > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > > replicas
> > > > > for
> > > > > > > a
> > > > > > > > > partition, not on the replicas for the
> > > > > __remote_log_segment_metadata
> > > > > > > > > partition. It's not clear how the leader of
> > > > > > > __remote_log_segment_metadata
> > > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > > >
> > > > > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> > > > > Could you
> > > > > > > > > update the KIP based on topicID?
> > > > > > > > >
> > > > > > > > > 5101. RLMM: It would be useful to clarify how the following
> > two
> > > > > APIs
> > > > > > > are
> > > > > > > > > used. According to the wiki, the former is used for topic
> > > deletion
> > > > > and
> > > > > > > the
> > > > > > > > > latter is used for retention. It seems that retention should
> > > use
> > > > > the
> > > > > > > former
> > > > > > > > > since remote segments without a matching epoch in the leader
> > > > > > > (potentially
> > > > > > > > > due to unclean leader election) also need to be garbage
> > > collected.
> > > > > The
> > > > > > > > > latter seems to be used for the new leader to determine the
> > > last
> > > > > tiered
> > > > > > > > > segment.
> > > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > > >
> > > > > > > > > 5102. RSM:
> > > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that
> > > they
> > > > > can
> > > > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead
> > of
> > > > > Long?
> > > > > > > > > 5102.3 Why only some of the methods have default
> > > implementation and
> > > > > > > others
> > > > > > > > > don't?
> > > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > > and DeletePartitionUpdate?
> > > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> > > > > since it
> > > > > > > will
> > > > > > > > > be generated in memory.
> > > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > > > > > baseOffset and
> > > > > > > > > startOffset. For example, deleteRecords() could move the
> > > > > startOffset
> > > > > > > to the
> > > > > > > > > middle of a segment. If we copy the full segment to remote
> > > > > storage, the
> > > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > > RemoteLogSegmentMetadata
> > > > > > > > > and LogSegmentData?
> > > > > > > > > 5102.8 Could we document whether endOffset in
> > > > > RemoteLogSegmentMetadata
> > > > > > > is
> > > > > > > > > inclusive/exclusive?
> > > > > > > > >
> > > > > > > > > 5103. configs:
> > > > > > > > > 5103.1 Could we define the default value of non-required
> > > configs
> > > > > (e.g
> > > > > > > the
> > > > > > > > > size of new thread pools)?
> > > > > > > > > 5103.2 It seems that local.log.retention.ms should default
> > to
> > > > > > > retention.ms,
> > > > > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > > > > that local.log.retention.bytes should default to
> > segment.bytes.
> > > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description
> > > says
> > > > > "used
> > > > > > > in
> > > > > > > > > scheduling tasks to copy segments, fetch remote log indexes
> > and
> > > > > clean
> > > > > > > up
> > > > > > > > > remote log segments". However, there is a separate
> > > > > > > > > config remote.log.reader.threads for fetching remote data.
> > It's
> > > > > weird
> > > > > > > to
> > > > > > > > > fetch remote index and log in different thread pools since
> > > both are
> > > > > > > used
> > > > > > > > > for serving fetch requests.
> > > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> > > amount of
> > > > > > > time to
> > > > > > > > > back off when there is no work to do? If so, perhaps it can
> > be
> > > > > renamed
> > > > > > > as
> > > > > > > > > backoff.ms.
> > > > > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > > > > configs?
> > > > > > > If
> > > > > > > > > so, they need to be listed in this section.
> > > > > > > > >
> > > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the
> > index
> > > > > files
> > > > > > > of
> > > > > > > > > remote log segments to avoid multiple index fetches from the
> > > remote
> > > > > > > > > storage." Is the RLM in memory or on disk? If on disk, where
> > > is it
> > > > > > > stored?
> > > > > > > > > Do we need a configuration to bound the size?
> > > > > > > > >
> > > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> > > > > Offset in
> > > > > > > > > different places. It would be useful to standardize the
> > > > > terminology.
> > > > > > > > >
> > > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> > > listed two
> > > > > > > options
> > > > > > > > > without saying which option is chosen.
> > > > > > > > >
> > > > > > > > > 5107. Follower to leader transition: It has step 2, but not
> > > step 1.
> > > > > > > > >
> > > > > > > > > 5108. If a consumer fetches from the remote data and the
> > remote
> > > > > > > storage is
> > > > > > > > > not available, what error code is used in the fetch response?
> > > > > > > > >
> > > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> > > > > message
> > > > > > > > > offset whose timestamp is >= to the given timestamp in the
> > > request.
> > > > > > > That
> > > > > > > > > means it checks in remote log time indexes first, after which
> > > > > local log
> > > > > > > > > time indexes are checked." Could you document which method in
> > > RLMM
> > > > > is
> > > > > > > used
> > > > > > > > > for this?
> > > > > > > > >
> > > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > > metadata of
> > > > > that
> > > > > > > > > partition with a delete marker and publishes them to RLMM."
> > > This
> > > > > seems
> > > > > > > > > outdated given the new topic deletion logic.
> > > > > > > > >
> > > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > > earliest
> > > > > leader
> > > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > > topicPartition,
> > > > > > > int
> > > > > > > > > leaderEpoch) and updates that as the log start offset." Do we
> > > need
> > > > > that
> > > > > > > > > since replication propagates logStartOffset already?
> > > > > > > > >
> > > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching
> > > from
> > > > > remote
> > > > > > > > > storage?
> > > > > > > > >
> > > > > > > > > 5113. "Committed offsets can be stored in a local file to
> > avoid
> > > > > > > reading the
> > > > > > > > > messages again when a broker is restarted." Could you
> > describe
> > > the
> > > > > > > format
> > > > > > > > > and the location of the file? Also, could the same message be
> > > > > > > processed by
> > > > > > > > > RLMM again after broker restart? If so, how do we handle
> > that?
> > > > > > > > >
> > > > > > > > > 5114. Message format
> > > > > > > > > 5114.1 There are two records named
> > > RemoteLogSegmentMetadataRecord
> > > > > with
> > > > > > > > > apiKey 0 and 1.
> > > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > > whether
> > > > > > > endOffset
> > > > > > > > > is inclusive/exclusive?
> > > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > > > LeaderEpoch a
> > > > > > > bit
> > > > > > > > > more? Is that the epoch of the leader when it copies the
> > > segment to
> > > > > > > remote
> > > > > > > > > storage? Also, how will this field be used?
> > > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more?
> > Each
> > > > > record
> > > > > > > in
> > > > > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > > >
> > > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the
> > > log
> > > > > > > cleaner
> > > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > > RemotePartitionRemover.
> > > > > > > > >
> > > > > > > > > 5116. "RLC receives the delete_partition_marked and processes
> > > it
> > > > > if it
> > > > > > > is
> > > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > > >
> > > > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > > > > metadata
> > > > > > > topic?
> > > > > > > > >
> > > > > > > > > 5118. "Maximum remote log reader thread pool task queue size.
> > > If
> > > > > the
> > > > > > > task
> > > > > > > > > queue is full, broker will stop reading remote log segments."
> > > What
> > > > > do
> > > > > > > we
> > > > > > > > > return to the fetch request in this case?
> > > > > > > > >
> > > > > > > > > 5119. It would be useful to list all things not supported in
> > > the
> > > > > first
> > > > > > > > > version in a Future work or Limitations section. For example,
> > > > > compacted
> > > > > > > > > topic, JBOD, changing remote.log.storage.enable from true to
> > > false,
> > > > > > > etc.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > > (
> > > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > >
> > > > > > >
> > > > > > >
> > > > >
> > >
> >

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

Posted by Jun Rao <ju...@confluent.io>.
Hi, Satish,

Thanks for the reply. A few more followup comments.

6022. For packages used for server plugins, the convention is to
use org.apache.kafka.server. See java-based Authorizer as an example.

9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat
file captures the state of the remote segments. After a partition is
deleted, it seems that we just need to remove the partitions's remote
segments from the flat file.

9101. Upgrade: It will be useful to allow direct upgrade from an old
version. It seems that's doable. One can just do the normal upgrade first
and wait enough time (for producer snapshots to be built), and then enable
remote storage.

9102. RemotePartitionRemover(RPM) process: Is it true that RPM starts
tracking the remote segments when RLMM.onPartitionLeadershipChanges() is
called with the broker being the leader for __remote_log_metadata
partition? If so, could we document it?

Jun

On Tue, Dec 15, 2020 at 8:47 AM Kowshik Prakasam <kp...@confluent.io>
wrote:

> Hi Satish,
>
> Thanks for the updates! A few more comments below.
>
> 9001. Under the "Upgrade" section, there is a line mentioning: "Upgrade the
> existing Kafka cluster to 2.7 version and allow this to run for the log
> retention of user topics that you want to enable tiered storage. This will
> allow all the topics to have the producer snapshots generated for each log
> segment." -- Which associated change in AK were you referring to here? Is
> it: https://github.com/apache/kafka/pull/7929 ? It seems like I don't see
> it in the 2.7 release branch yet, here is the link:
> https://github.com/apache/kafka/commits/2.7.
>
> 9002. Under the "Upgrade" section, the configuration mentioned is
> 'remote.log.storage.system.enable'. However, under "Public Interfaces"
> section the corresponding configuration is 'remote.storage.system.enable'.
> Could we use the same one in both, maybe
> 'remote.log.storage.system.enable'?
>
> 9003. Under "Per Topic Configuration", the KIP recommends setting
> 'remote.log.storage.enable' to true at a per-topic level. It will be useful
> to add a line that if the user wants to enable it for all topics, then they
> should be able to set the cluster-wide default to true. Also, it will be
> useful to mention that the KIP currently does not support setting it to
> false (after it is set to true), and add that to the future work section.
>
> 9004. Under "Committed offsets file format", the sample provided shows
> partition number and offset. Is the topic name required for identifying
> which topic the partitions belong to?
>
> 9005. Under "Internal flat-file store format of remote log metadata", it
> seems useful to specify both topic name and topic ID for debugging
> purposes.
>
> 9006. Under "Internal flat-file store format of remote log metadata", the
> description of "metadata-topic-offset" currently says "offset of the remote
> log metadata topic from which this topic partition's remote log metadata is
> fetched." Just for the wording, perhaps you meant to refer to the offset
> upto which the file has been committed? i.e. "offset of the remote log
> metadata topic upto which this topic partition's remote log metadata has
> been committed into this file."
>
> 9007. Under "Internal flat-file store format of remote log metadata", the
> schema of the payload (i.e. beyond the header) seems to contain the events
> from the metadata topic. It seems useful to instead persist the
> representation of the materialized state of the events, so that for the
> same segment only the latest state is stored. Besides reducing storage
> footprint, this also is likely to relate directly with the in-memory
> representation of the RLMM cache (which probably is some kind of a Map with
> key being segment ID and value being the segment state), so recovery from
> disk will be straightforward.
>
> 9008. Under "Topic deletion lifecycle", step (1), it will be useful to
> mention when in the deletion flow does the controller publish the
> delete_partition_marked event to say that the partition is marked for
> deletion?
>
> 9009. There are ~4 TODOs in the KIP. Could you please address these or
> remove them?
>
> 9010. There is a reference to a Google doc on the KIP which was used
> earlier for discussions. Please could you remove the reference, since the
> KIP is the source of the truth?
>
> 9011. This feedback is from an earlier comment. 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. What do you think?
>
>
> Cheers,
> Kowshik
>
>
> On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi Jun,
> > Thanks for your comments. Please go through the inline replies.
> >
> >
> > 5102.2: It seems that both positions can just be int. Another option is
> to
> > have two methods. Would it be clearer?
> >
> >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata,  int startPosition)
> throwsRemoteStorageException;
> >
> >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata, int startPosition, int endPosition) throws
> > RemoteStorageException;
> >
> > That makes sense to me, updated the KIP.
> >
> > 6003: Could you also update the javadoc for the return value?
> >
> > Updated.
> >
> > 6020: local.log.retention.bytes: Should it default to log.retention.bytes
> > to be consistent with local.log.retention.ms?
> >
> > Yes, it can be defaulted to log.retention.bytes.
> >
> > 6021: Could you define TopicIdPartition?
> >
> > Added TopicIdPartition in the KIP.
> >
> > 6022: For all public facing classes, could you specify the package name?
> >
> > Updated.
> >
> >
> > Thanks,
> > Satish.
> >
> > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Hi, Satish,
> > >
> > > Thanks for the reply. A few more comments below.
> > >
> > > 5102.2: It seems that both positions can just be int. Another option is
> > to
> > > have two methods. Would it be clearer?
> > >
> > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > remoteLogSegmentMetadata,
> > >                                     int startPosition) throws
> > > RemoteStorageException;
> > >
> > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > remoteLogSegmentMetadata,
> > >                                     int startPosition, int endPosition)
> > > throws RemoteStorageException;
> > >
> > > 6003: Could you also update the javadoc for the return value?
> > >
> > > 6010: What kind of tiering throughput have you seen with 5 threads?
> > >
> > > 6020: local.log.retention.bytes: Should it default to
> log.retention.bytes
> > > to be consistent with local.log.retention.ms?
> > >
> > > 6021: Could you define TopicIdPartition?
> > >
> > > 6022: For all public facing classes, could you specify the package
> name?
> > >
> > > It seems that you already added the topicId support. Two other
> remaining
> > > items are (a) the format of local tier metadata storage and (b)
> upgrade.
> > >
> > > Jun
> > >
> > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <
> satish.duggana@gmail.com>
> > > wrote:
> > >
> > > > Hi Jun,
> > > > Thanks for your comments. Please find the inline replies below.
> > > >
> > > > >605.2 It's rare for the follower to need the remote data. So, the
> > current
> > > > approach is fine too. Could you document the process of rebuilding
> the
> > > > producer state since we can't simply trim the producerState to an
> > offset in
> > > > the middle of a segment.
> > > >
> > > > Will clarify in the KIP.
> > > >
> > > > >5102.2 Would it be clearer to make startPosiont long and endPosition
> > of
> > > > Optional<Long>?
> > > >
> > > > We will have arg checks with respective validation. It is not a good
> > > > practice to have arguments with optional as mentioned here.
> > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > >
> > > >
> > > > >5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > > ByteBuffer.
> > > >
> > > > Updated.
> > > >
> > > > >5102.7 Could you define all public methods for LogSegmentData?
> > > >
> > > > Updated.
> > > >
> > > > >5103.5 Could you change the reference to rlm_process_interval_ms and
> > > > rlm_retry_interval_ms to the new config names? Also, the retry
> interval
> > > > config seems still missing. It would be useful to support exponential
> > > > backoff with the retry interval config.
> > > >
> > > > Good point. We wanted the retry with truncated exponential backoff,
> > > > updated the KIP.
> > > >
> > > > >5111. "RLM follower fetches the earliest offset for the earliest
> > leader
> > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> topicPartition,
> > int
> > > > leaderEpoch) and updates that as the log start offset." This text is
> > still
> > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > >
> > > > Updated.
> > > >
> > > > >5115. There are still references to "remote log cleaners".
> > > >
> > > > Updated.
> > > >
> > > > >6000. Since we are returning new error codes, we need to bump up the
> > > > protocol version for Fetch request. Also, it will be useful to
> > document all
> > > > new error codes and whether they are retriable or not.
> > > >
> > > > Sure, we will add that in the KIP.
> > > >
> > > > >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > leaderEpoch
> > > > is int32 instead of long.
> > > >
> > > > Updated.
> > > >
> > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > > RemoteLogSegmentMetadata.state()?
> > > >
> > > > No, it is fixed.
> > > >
> > > > >6003. RemoteLogSegmentMetadata
> remoteLogSegmentMetadata(TopicPartition
> > > > topicPartition, long offset, int epochForOffset): Should this return
> > > > Optional<RemoteLogSegmentMetadata>?
> > > >
> > > > That makes sense, updated.
> > > >
> > > > >6005. RemoteLogState: It seems it's better to split it between
> > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the
> > states
> > > > are never shared between the two use cases.
> > > >
> > > > Agree with that, updated.
> > > >
> > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However,
> > is it
> > > > ture that other than the metadata topic, RLMM just needs to know
> > whether
> > > > there is a replica assigned to this broker and doesn't need to know
> > whether
> > > > the replica is the leader or the follower?
> > > >
> > > > That may be true. If the implementation does not need that, it can
> > > > ignore the information in the callback.
> > > >
> > > > >6007: "Handle expired remote segments (leader and follower)": Why is
> > this
> > > > needed in both the leader and the follower?
> > > >
> > > > Updated.
> > > >
> > > > >6008.       "name": "SegmentSizeInBytes",
> > > >                 "type": "int64",
> > > > The segment size can just be int32.
> > > >
> > > > Updated.
> > > >
> > > > >6009. For the record format in the log, it seems that we need to add
> > > > record
> > > > type and record version before the serialized bytes. We can follow
> the
> > > > convention used in
> > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > >
> > > > Yes, KIP already mentions that these are serialized before the
> payload
> > > > as below. We will mention explicitly that these two are written
> before
> > > > the data is written.
> > > >
> > > > RLMM instance on broker publishes the message to the topic with key
> as
> > > > null and value with the below format.
> > > >
> > > > type      : unsigned var int, represents the value type. This value
> is
> > > > 'apikey' as mentioned in the schema.
> > > > version : unsigned var int, the 'version' number of the type as
> > > > mentioned in the schema.
> > > > data      : record payload in kafka protocol message format.
> > > >
> > > >
> > > > >6010. remote.log.manager.thread.pool.size: The default value is 10.
> > This
> > > > might be too high when enabling the tiered feature for the first
> time.
> > > > Since there are lots of segments that need to be tiered initially, a
> > large
> > > > number of threads could overwhelm the broker.
> > > >
> > > > Is the default value 5 reasonable?
> > > >
> > > > 6011. "The number of milli seconds to keep the local log segment
> > before it
> > > > gets deleted. If not set, the value in `log.retention.minutes` is
> > used. If
> > > > set to -1, no time limit is applied." We should use log.retention.ms
> > > > instead of log.retention.minutes.
> > > > Nice typo catch. Updated the KIP.
> > > >
> > > > Thanks,
> > > > Satish.
> > > >
> > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the updated KIP. A few more comments below.
> > > > >
> > > > > 605.2 It's rare for the follower to need the remote data. So, the
> > current
> > > > > approach is fine too. Could you document the process of rebuilding
> > the
> > > > > producer state since we can't simply trim the producerState to an
> > offset
> > > > in
> > > > > the middle of a segment.
> > > > >
> > > > > 5102.2 Would it be clearer to make startPosiont long and
> endPosition
> > of
> > > > > Optional<Long>?
> > > > >
> > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > > > ByteBuffer.
> > > > >
> > > > > 5102.7 Could you define all public methods for LogSegmentData?
> > > > >
> > > > > 5103.5 Could you change the reference to rlm_process_interval_ms
> and
> > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > interval
> > > > > config seems still missing. It would be useful to support
> exponential
> > > > > backoff with the retry interval config.
> > > > >
> > > > > 5111. "RLM follower fetches the earliest offset for the earliest
> > leader
> > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > topicPartition,
> > > > int
> > > > > leaderEpoch) and updates that as the log start offset." This text
> is
> > > > still
> > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > >
> > > > > 5115. There are still references to "remote log cleaners".
> > > > >
> > > > > 6000. Since we are returning new error codes, we need to bump up
> the
> > > > > protocol version for Fetch request. Also, it will be useful to
> > document
> > > > all
> > > > > new error codes and whether they are retriable or not.
> > > > >
> > > > > 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > > leaderEpoch
> > > > > is int32 instead of long.
> > > > >
> > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > > > RemoteLogSegmentMetadata.state()?
> > > > >
> > > > > 6003. RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata(TopicPartition
> > > > > topicPartition, long offset, int epochForOffset): Should this
> return
> > > > > Optional<RemoteLogSegmentMetadata>?
> > > > >
> > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to pick a
> > more
> > > > > indicative name so that people understand what epoch this is.
> > > > >
> > > > > 6005. RemoteLogState: It seems it's better to split it between
> > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the
> > states
> > > > > are never shared between the two use cases.
> > > > >
> > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However,
> > is it
> > > > > ture that other than the metadata topic, RLMM just needs to know
> > whether
> > > > > there is a replica assigned to this broker and doesn't need to know
> > > > whether
> > > > > the replica is the leader or the follower?
> > > > >
> > > > > 6007: "Handle expired remote segments (leader and follower)": Why
> is
> > this
> > > > > needed in both the leader and the follower?
> > > > >
> > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > >                 "type": "int64",
> > > > > The segment size can just be int32.
> > > > >
> > > > > 6009. For the record format in the log, it seems that we need to
> add
> > > > record
> > > > > type and record version before the serialized bytes. We can follow
> > the
> > > > > convention used in
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > .
> > > > >
> > > > > 6010. remote.log.manager.thread.pool.size: The default value is 10.
> > This
> > > > > might be too high when enabling the tiered feature for the first
> > time.
> > > > > Since there are lots of segments that need to be tiered initially,
> a
> > > > large
> > > > > number of threads could overwhelm the broker.
> > > > >
> > > > > 6011. "The number of milli seconds to keep the local log segment
> > before
> > > > it
> > > > > gets deleted. If not set, the value in `log.retention.minutes` is
> > used.
> > > > If
> > > > > set to -1, no time limit is applied." We should use
> log.retention.ms
> > > > > instead of log.retention.minutes.
> > > > >
> > > > > Jun
> > > > >
> > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > satish.duggana@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi,
> > > > > > We updated the KIP with the points mentioned in the earlier mail
> > > > > > except for KIP-516 related changes. You can go through them and
> > let us
> > > > > > know if you have any comments. We will update the KIP with the
> > > > > > remaining todo items and KIP-516 related changes by end of this
> > > > > > week(5th Dec).
> > > > > >
> > > > > > Thanks,
> > > > > > Satish.
> > > > > >
> > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > Hi Jun,
> > > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > > >
> > > > > > > 605.2 "Build the local leader epoch cache by cutting the leader
> > epoch
> > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > mentioned an
> > > > > > issue
> > > > > > > earlier. Suppose the leader's local start offset is 100. The
> > follower
> > > > > > finds
> > > > > > > a remote segment covering offset range [80, 120). The
> > producerState
> > > > with
> > > > > > > this remote segment is up to offset 120. To trim the
> > producerState to
> > > > > > > offset 100 requires more work since one needs to download the
> > > > previous
> > > > > > > producerState up to offset 80 and then replay the messages from
> > 80 to
> > > > > > 100.
> > > > > > > It seems that it's simpler in this case for the follower just
> to
> > > > take the
> > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > >
> > > > > > > We chose that approach to avoid any edge cases here. It may be
> > > > > > > possible that the remote log segment that is received may not
> > have
> > > > the
> > > > > > > same leader epoch sequence from 100-120 as it contains on the
> > > > > > > leader(this can happen due to unclean leader). It is safe to
> > start
> > > > > > > from what the leader returns here.Another way is to find the
> > remote
> > > > > > > log segment
> > > > > > >
> > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > replicas
> > > > for a
> > > > > > > partition, not on the replicas for the
> > __remote_log_segment_metadata
> > > > > > > partition. It's not clear how the leader of
> > > > __remote_log_segment_metadata
> > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > >
> > > > > > > RLMM will always receive the callback for the remote log
> metadata
> > > > > > > topic partitions hosted on the local broker and these will be
> > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > >
> > > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> > Could
> > > > you
> > > > > > > update the KIP based on topicID?
> > > > > > >
> > > > > > > We mentioned KIP-516 and how it helps. We will update this KIP
> > with
> > > > > > > all the changes it brings with KIP-516.
> > > > > > >
> > > > > > > 5101. RLMM: It would be useful to clarify how the following two
> > APIs
> > > > are
> > > > > > > used. According to the wiki, the former is used for topic
> > deletion
> > > > and
> > > > > > the
> > > > > > > latter is used for retention. It seems that retention should
> use
> > the
> > > > > > former
> > > > > > > since remote segments without a matching epoch in the leader
> > > > (potentially
> > > > > > > due to unclean leader election) also need to be garbage
> > collected.
> > > > The
> > > > > > > latter seems to be used for the new leader to determine the
> last
> > > > tiered
> > > > > > > segment.
> > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > topicPartition, long leaderEpoch);
> > > > > > >
> > > > > > > Right,.that is what we are currently doing. We will update the
> > > > > > > javadocs and wiki with that. Earlier, we did not want to remove
> > the
> > > > > > > segments which are not matched with leader epochs from the
> ladder
> > > > > > > partition as they may be used later by a replica which can
> > become a
> > > > > > > leader (unclean leader election) and refer those segments. But
> > that
> > > > > > > may leak these segments in remote storage until the topic
> > lifetime.
> > > > We
> > > > > > > decided to cleanup the segments with the oldest incase of size
> > based
> > > > > > > retention also.
> > > > > > >
> > > > > > > 5102. RSM:
> > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that
> > they can
> > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > >
> > > > > > > It will be useful to have metadata for RSM to fetch log
> segment.
> > It
> > > > > > > may create location/path using id with other metadata too.
> > > > > > >
> > > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of
> > Long?
> > > > > > >
> > > > > > > Wanted to keep endPosition as optional to read till the end of
> > the
> > > > > > > segment and avoid sentinels.
> > > > > > >
> > > > > > > 5102.3 Why only some of the methods have default implementation
> > and
> > > > > > others
> > > > > > > Don't?
> > > > > > >
> > > > > > > Actually,  RSM will not have any default implementations.
> Those 3
> > > > > > > methods were made default earlier for tests etc. Updated the
> > wiki.
> > > > > > >
> > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > and DeletePartitionUpdate?
> > > > > > >
> > > > > > > Sure, they will be added.
> > > > > > >
> > > > > > >
> > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> > since
> > > > it
> > > > > > will
> > > > > > > be generated in memory.
> > > > > > >
> > > > > > > Right, this is in plan.
> > > > > > >
> > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > > baseOffset
> > > > > > and
> > > > > > > startOffset. For example, deleteRecords() could move the
> > startOffset
> > > > to
> > > > > > the
> > > > > > > middle of a segment. If we copy the full segment to remote
> > storage,
> > > > the
> > > > > > > baseOffset and the startOffset will be different.
> > > > > > >
> > > > > > > Good point. startOffset is baseOffset by default, if not set
> > > > explicitly.
> > > > > > >
> > > > > > > 5102.7 Could we define all the public methods for
> > > > > > RemoteLogSegmentMetadata
> > > > > > > and LogSegmentData?
> > > > > > >
> > > > > > > Sure, updated the wiki.
> > > > > > >
> > > > > > > 5102.8 Could we document whether endOffset in
> > > > RemoteLogSegmentMetadata is
> > > > > > > inclusive/exclusive?
> > > > > > >
> > > > > > > It is inclusive, will update.
> > > > > > >
> > > > > > > 5103. configs:
> > > > > > > 5103.1 Could we define the default value of non-required
> configs
> > > > (e.g the
> > > > > > > size of new thread pools)?
> > > > > > >
> > > > > > > Sure, that makes sense.
> > > > > > >
> > > > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > > > > retention.ms,
> > > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > > > >
> > > > > > > Right, we do not have  remote.log.retention as we discussed
> > earlier.
> > > > > > > Thanks for catching the typo.
> > > > > > >
> > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description
> says
> > > > "used in
> > > > > > > scheduling tasks to copy segments, fetch remote log indexes and
> > > > clean up
> > > > > > > remote log segments". However, there is a separate
> > > > > > > config remote.log.reader.threads for fetching remote data. It's
> > > > weird to
> > > > > > > fetch remote index and log in different thread pools since both
> > are
> > > > used
> > > > > > > for serving fetch requests.
> > > > > > >
> > > > > > > Right, remote.log.manager.thread.pool is mainly used for
> > copy/cleanup
> > > > > > > activities. Fetch path always goes through
> > remote.log.reader.threads.
> > > > > > >
> > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount
> > of
> > > > time
> > > > > > to
> > > > > > > back off when there is no work to do? If so, perhaps it can be
> > > > renamed as
> > > > > > > backoff.ms.
> > > > > > >
> > > > > > > This is the delay interval for each iteration. It may be
> renamed
> > to
> > > > > > > remote.log.manager.task.delay.ms
> > > > > > >
> > > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > > > configs? If
> > > > > > > so, they need to be listed in this section.
> > > > > > >
> > > > > > > remote.log.manager.task.interval.ms is the process internal,
> > retry
> > > > > > > interval is missing in the configs, which will be updated in
> the
> > KIP.
> > > > > > >
> > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> > > > files of
> > > > > > > remote log segments to avoid multiple index fetches from the
> > remote
> > > > > > > storage." Is the RLM in memory or on disk? If on disk, where is
> > it
> > > > > > stored?
> > > > > > > Do we need a configuration to bound the size?
> > > > > > >
> > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > `remote-log-index-cache` under log dir. We plan to have a
> config
> > for
> > > > > > > that instead of default. We will have a configuration for that.
> > > > > > >
> > > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> > Offset
> > > > in
> > > > > > > different places. It would be useful to standardize the
> > terminology.
> > > > > > >
> > > > > > > Sure.
> > > > > > >
> > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It listed
> > two
> > > > > > options
> > > > > > > without saying which option is chosen.
> > > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > > >
> > > > > > > 5107. Follower to leader transition: It has step 2, but not
> step
> > 1.
> > > > > > > Step-1 is there but it is not explicitly highlighted. It is
> > previous
> > > > > > > table to step-2.
> > > > > > >
> > > > > > > 5108. If a consumer fetches from the remote data and the remote
> > > > storage
> > > > > > is
> > > > > > > not available, what error code is used in the fetch response?
> > > > > > >
> > > > > > > Good point. We have not yet defined the error for this case. We
> > need
> > > > > > > to define an error message and send the same in fetch response.
> > > > > > >
> > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> > message
> > > > > > > offset whose timestamp is >= to the given timestamp in the
> > request.
> > > > That
> > > > > > > means it checks in remote log time indexes first, after which
> > local
> > > > log
> > > > > > > time indexes are checked." Could you document which method in
> > RLMM is
> > > > > > used
> > > > > > > for this?
> > > > > > >
> > > > > > > Okay.
> > > > > > >
> > > > > > > 5110. Stopreplica: "it sets all the remote log segment metadata
> > of
> > > > that
> > > > > > > partition with a delete marker and publishes them to RLMM."
> This
> > > > seems
> > > > > > > outdated given the new topic deletion logic.
> > > > > > >
> > > > > > > Will update with KIP-516 related points.
> > > > > > >
> > > > > > > 5111. "RLM follower fetches the earliest offset for the
> earliest
> > > > leader
> > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > topicPartition,
> > > > > > int
> > > > > > > leaderEpoch) and updates that as the log start offset." Do we
> > need
> > > > that
> > > > > > > since replication propagates logStartOffset already?
> > > > > > >
> > > > > > > Good point. Right, existing replication protocol takes care of
> > > > > > > updating the followers’s log start offset received from the
> > leader.
> > > > > > >
> > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching
> from
> > > > remote
> > > > > > > storage?
> > > > > > >
> > > > > > > Remote reads may fail within the current default wait time, but
> > > > > > > subsequent fetches would be able to serve as that data is
> stored
> > in
> > > > > > > the local cache. This cache is currently implemented in RSMs.
> > But we
> > > > > > > plan to pull this into the remote log messaging layer in
> future.
> > > > > > >
> > > > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > > > reading
> > > > > > the
> > > > > > > messages again when a broker is restarted." Could you describe
> > the
> > > > format
> > > > > > > and the location of the file? Also, could the same message be
> > > > processed
> > > > > > by
> > > > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > > > >
> > > > > > > Sure, we will update in the KIP.
> > > > > > >
> > > > > > > 5114. Message format
> > > > > > > 5114.1 There are two records named
> RemoteLogSegmentMetadataRecord
> > > > with
> > > > > > > apiKey 0 and 1.
> > > > > > >
> > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > >
> > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> whether
> > > > > > endOffset
> > > > > > > is inclusive/exclusive?
> > > > > > > It is inclusive, will update.
> > > > > > >
> > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > LeaderEpoch
> > > > a
> > > > > > bit
> > > > > > > more? Is that the epoch of the leader when it copies the
> segment
> > to
> > > > > > remote
> > > > > > > storage? Also, how will this field be used?
> > > > > > >
> > > > > > > Right, this is the leader epoch of the broker which copied this
> > > > > > > segment. This is helpful in reason about which broker copied
> the
> > > > > > > segment to remote storage.
> > > > > > >
> > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> > > > record in
> > > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > > >
> > > > > > > This is the  timestamp at which  the respective event occurred.
> > Added
> > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > > > > > implementation. We thought about that but it looked cleaner to
> > use at
> > > > > > > the message structure level instead of getting that from the
> > consumer
> > > > > > > record and using that to build the respective event.
> > > > > > >
> > > > > > >
> > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > >
> > > > > > > Right, it looks like config allows only int value >= 14.
> > > > > > >
> > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the
> log
> > > > cleaner
> > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > RemotePartitionRemover.
> > > > > > >
> > > > > > > I am fine with RemotePartitionRemover or
> > RemoteLogDeletionManager(we
> > > > > > > have other manager classes like RLM, RLMM).
> > > > > > >
> > > > > > > 5116. "RLC receives the delete_partition_marked and processes
> it
> > if
> > > > it is
> > > > > > > not yet processed earlier." How does it know whether
> > > > > > > delete_partition_marked has been processed earlier?
> > > > > > >
> > > > > > > This is to handle duplicate delete_partition_marked events. RLC
> > > > > > > internally maintains a state for the delete_partition events
> and
> > if
> > > > it
> > > > > > > already has an existing event then it ignores if it is already
> > being
> > > > > > > processed.
> > > > > > >
> > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > metadata
> > > > > > topic?
> > > > > > >
> > > > > > > Right, this is in plan but did not mention it in the KIP. This
> > will
> > > > be
> > > > > > > useful for debugging purposes too.
> > > > > > >
> > > > > > > 5118. "Maximum remote log reader thread pool task queue size.
> If
> > the
> > > > task
> > > > > > > queue is full, broker will stop reading remote log segments."
> > What
> > > > do we
> > > > > > > return to the fetch request in this case?
> > > > > > >
> > > > > > > We return an error response for that partition.
> > > > > > >
> > > > > > > 5119. It would be useful to list all things not supported in
> the
> > > > first
> > > > > > > version in a Future work or Limitations section. For example,
> > > > compacted
> > > > > > > topic, JBOD, changing remote.log.storage.enable from true to
> > false,
> > > > etc.
> > > > > > >
> > > > > > > We already have a non-goals section which is filled with some
> of
> > > > these
> > > > > > > details. Do we need another limitations section?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > >
> > > > > > > > 605.2 "Build the local leader epoch cache by cutting the
> leader
> > > > epoch
> > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > mentioned
> > > > an
> > > > > > issue
> > > > > > > > earlier. Suppose the leader's local start offset is 100. The
> > > > follower
> > > > > > finds
> > > > > > > > a remote segment covering offset range [80, 120). The
> > producerState
> > > > > > with
> > > > > > > > this remote segment is up to offset 120. To trim the
> > producerState
> > > > to
> > > > > > > > offset 100 requires more work since one needs to download the
> > > > previous
> > > > > > > > producerState up to offset 80 and then replay the messages
> > from 80
> > > > to
> > > > > > 100.
> > > > > > > > It seems that it's simpler in this case for the follower just
> > to
> > > > take
> > > > > > the
> > > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > > >
> > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > replicas
> > > > for
> > > > > > a
> > > > > > > > partition, not on the replicas for the
> > > > __remote_log_segment_metadata
> > > > > > > > partition. It's not clear how the leader of
> > > > > > __remote_log_segment_metadata
> > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > >
> > > > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> > > > Could you
> > > > > > > > update the KIP based on topicID?
> > > > > > > >
> > > > > > > > 5101. RLMM: It would be useful to clarify how the following
> two
> > > > APIs
> > > > > > are
> > > > > > > > used. According to the wiki, the former is used for topic
> > deletion
> > > > and
> > > > > > the
> > > > > > > > latter is used for retention. It seems that retention should
> > use
> > > > the
> > > > > > former
> > > > > > > > since remote segments without a matching epoch in the leader
> > > > > > (potentially
> > > > > > > > due to unclean leader election) also need to be garbage
> > collected.
> > > > The
> > > > > > > > latter seems to be used for the new leader to determine the
> > last
> > > > tiered
> > > > > > > > segment.
> > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > >
> > > > > > > > 5102. RSM:
> > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that
> > they
> > > > can
> > > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead
> of
> > > > Long?
> > > > > > > > 5102.3 Why only some of the methods have default
> > implementation and
> > > > > > others
> > > > > > > > don't?
> > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > and DeletePartitionUpdate?
> > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> > > > since it
> > > > > > will
> > > > > > > > be generated in memory.
> > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > > > > baseOffset and
> > > > > > > > startOffset. For example, deleteRecords() could move the
> > > > startOffset
> > > > > > to the
> > > > > > > > middle of a segment. If we copy the full segment to remote
> > > > storage, the
> > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > RemoteLogSegmentMetadata
> > > > > > > > and LogSegmentData?
> > > > > > > > 5102.8 Could we document whether endOffset in
> > > > RemoteLogSegmentMetadata
> > > > > > is
> > > > > > > > inclusive/exclusive?
> > > > > > > >
> > > > > > > > 5103. configs:
> > > > > > > > 5103.1 Could we define the default value of non-required
> > configs
> > > > (e.g
> > > > > > the
> > > > > > > > size of new thread pools)?
> > > > > > > > 5103.2 It seems that local.log.retention.ms should default
> to
> > > > > > retention.ms,
> > > > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > > > that local.log.retention.bytes should default to
> segment.bytes.
> > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description
> > says
> > > > "used
> > > > > > in
> > > > > > > > scheduling tasks to copy segments, fetch remote log indexes
> and
> > > > clean
> > > > > > up
> > > > > > > > remote log segments". However, there is a separate
> > > > > > > > config remote.log.reader.threads for fetching remote data.
> It's
> > > > weird
> > > > > > to
> > > > > > > > fetch remote index and log in different thread pools since
> > both are
> > > > > > used
> > > > > > > > for serving fetch requests.
> > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> > amount of
> > > > > > time to
> > > > > > > > back off when there is no work to do? If so, perhaps it can
> be
> > > > renamed
> > > > > > as
> > > > > > > > backoff.ms.
> > > > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > > > configs?
> > > > > > If
> > > > > > > > so, they need to be listed in this section.
> > > > > > > >
> > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the
> index
> > > > files
> > > > > > of
> > > > > > > > remote log segments to avoid multiple index fetches from the
> > remote
> > > > > > > > storage." Is the RLM in memory or on disk? If on disk, where
> > is it
> > > > > > stored?
> > > > > > > > Do we need a configuration to bound the size?
> > > > > > > >
> > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> > > > Offset in
> > > > > > > > different places. It would be useful to standardize the
> > > > terminology.
> > > > > > > >
> > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> > listed two
> > > > > > options
> > > > > > > > without saying which option is chosen.
> > > > > > > >
> > > > > > > > 5107. Follower to leader transition: It has step 2, but not
> > step 1.
> > > > > > > >
> > > > > > > > 5108. If a consumer fetches from the remote data and the
> remote
> > > > > > storage is
> > > > > > > > not available, what error code is used in the fetch response?
> > > > > > > >
> > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> > > > message
> > > > > > > > offset whose timestamp is >= to the given timestamp in the
> > request.
> > > > > > That
> > > > > > > > means it checks in remote log time indexes first, after which
> > > > local log
> > > > > > > > time indexes are checked." Could you document which method in
> > RLMM
> > > > is
> > > > > > used
> > > > > > > > for this?
> > > > > > > >
> > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > metadata of
> > > > that
> > > > > > > > partition with a delete marker and publishes them to RLMM."
> > This
> > > > seems
> > > > > > > > outdated given the new topic deletion logic.
> > > > > > > >
> > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > earliest
> > > > leader
> > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > topicPartition,
> > > > > > int
> > > > > > > > leaderEpoch) and updates that as the log start offset." Do we
> > need
> > > > that
> > > > > > > > since replication propagates logStartOffset already?
> > > > > > > >
> > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching
> > from
> > > > remote
> > > > > > > > storage?
> > > > > > > >
> > > > > > > > 5113. "Committed offsets can be stored in a local file to
> avoid
> > > > > > reading the
> > > > > > > > messages again when a broker is restarted." Could you
> describe
> > the
> > > > > > format
> > > > > > > > and the location of the file? Also, could the same message be
> > > > > > processed by
> > > > > > > > RLMM again after broker restart? If so, how do we handle
> that?
> > > > > > > >
> > > > > > > > 5114. Message format
> > > > > > > > 5114.1 There are two records named
> > RemoteLogSegmentMetadataRecord
> > > > with
> > > > > > > > apiKey 0 and 1.
> > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > whether
> > > > > > endOffset
> > > > > > > > is inclusive/exclusive?
> > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > > LeaderEpoch a
> > > > > > bit
> > > > > > > > more? Is that the epoch of the leader when it copies the
> > segment to
> > > > > > remote
> > > > > > > > storage? Also, how will this field be used?
> > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more?
> Each
> > > > record
> > > > > > in
> > > > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > >
> > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the
> > log
> > > > > > cleaner
> > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > RemotePartitionRemover.
> > > > > > > >
> > > > > > > > 5116. "RLC receives the delete_partition_marked and processes
> > it
> > > > if it
> > > > > > is
> > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > >
> > > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > > > metadata
> > > > > > topic?
> > > > > > > >
> > > > > > > > 5118. "Maximum remote log reader thread pool task queue size.
> > If
> > > > the
> > > > > > task
> > > > > > > > queue is full, broker will stop reading remote log segments."
> > What
> > > > do
> > > > > > we
> > > > > > > > return to the fetch request in this case?
> > > > > > > >
> > > > > > > > 5119. It would be useful to list all things not supported in
> > the
> > > > first
> > > > > > > > version in a Future work or Limitations section. For example,
> > > > compacted
> > > > > > > > topic, JBOD, changing remote.log.storage.enable from true to
> > false,
> > > > > > etc.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > (
> > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > >>>>>>>
> > > > > > > > > >
> > > > > >
> > > > > >
> > > >
> >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Kowshik,
Thanks for your comments. Please find the inline replies below.

9002. Under the "Upgrade" section, the configuration mentioned is
'remote.log.storage.system.enable'. However, under "Public Interfaces"
section the corresponding configuration is 'remote.storage.system.enable'.
Could we use the same one in both, maybe 'remote.log.storage.system.enable'?

Nice catch, updated the KIP.

9003. Under "Per Topic Configuration", the KIP recommends setting
'remote.log.storage.enable' to true at a per-topic level. It will be useful
to add a line that if the user wants to enable it for all topics, then they
should be able to set the cluster-wide default to true. Also, it will be
useful to mention that the KIP currently does not support setting it to
false (after it is set to true), and add that to the future work section.

We do not want to expose a config at cluster level in the initial
version. We will add that in the future. Both limitations are added to
future work.


9004. Under "Committed offsets file format", the sample provided shows
partition number and offset. Is the topic name required for identifying
which topic the partitions belong to?
File name is already mentioned as _rlmm_committed_offsets‘’ and we
already know the internal remote log metadata topic name and it is
never going to be changed.

9005. Under "Internal flat-file store format of remote log metadata", it
seems useful to specify both topic name and topic ID for debugging
purposes.

That makes sense, updated.

9006. Under "Internal flat-file store format of remote log metadata", the
description of "metadata-topic-offset" currently says "offset of the remote
log metadata topic from which this topic partition's remote log metadata is
fetched." Just for the wording, perhaps you meant to refer to the offset
upto which the file has been committed? i.e. "offset of the remote log
metadata topic upto which this topic partition's remote log metadata has
been committed into this file."
Updated

9007. Under "Internal flat-file store format of remote log metadata", the
schema of the payload (i.e. beyond the header) seems to contain the events
from the metadata topic. It seems useful to instead persist the
representation of the materialized state of the events, so that for the
same segment only the latest state is stored. Besides reducing storage
footprint, this also is likely to relate directly with the in-memory
representation of the RLMM cache (which probably is some kind of a Map with
key being segment ID and value being the segment state), so recovery from
disk will be straightforward.

This is what we already do and clarified  in the earlier meeting..

9008. Under "Topic deletion lifecycle", step (1), it will be useful to
mention when in the deletion flow does the controller publish the
delete_partition_marked event to say that the partition is marked for
deletion?
Updated.

9009. There are ~4 TODOs in the KIP. Could you please address these or
remove them?
Updated.

9010. There is a reference to a Google doc on the KIP which was used
earlier for discussions. Please could you remove the reference, since the
KIP is the source of the truth?

Which doc reference are you saying?

9011. This feedback is from an earlier comment. 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. What do you think?

Sure, updated in the KIP.



On Tue, 15 Dec 2020 at 22:17, Kowshik Prakasam <kp...@confluent.io> wrote:
>
> Hi Satish,
>
> Thanks for the updates! A few more comments below.
>
> 9001. Under the "Upgrade" section, there is a line mentioning: "Upgrade the
> existing Kafka cluster to 2.7 version and allow this to run for the log
> retention of user topics that you want to enable tiered storage. This will
> allow all the topics to have the producer snapshots generated for each log
> segment." -- Which associated change in AK were you referring to here? Is
> it: https://github.com/apache/kafka/pull/7929 ? It seems like I don't see
> it in the 2.7 release branch yet, here is the link:
> https://github.com/apache/kafka/commits/2.7.
>
> 9002. Under the "Upgrade" section, the configuration mentioned is
> 'remote.log.storage.system.enable'. However, under "Public Interfaces"
> section the corresponding configuration is 'remote.storage.system.enable'.
> Could we use the same one in both, maybe 'remote.log.storage.system.enable'?
>
> 9003. Under "Per Topic Configuration", the KIP recommends setting
> 'remote.log.storage.enable' to true at a per-topic level. It will be useful
> to add a line that if the user wants to enable it for all topics, then they
> should be able to set the cluster-wide default to true. Also, it will be
> useful to mention that the KIP currently does not support setting it to
> false (after it is set to true), and add that to the future work section.
>
> 9004. Under "Committed offsets file format", the sample provided shows
> partition number and offset. Is the topic name required for identifying
> which topic the partitions belong to?
>
> 9005. Under "Internal flat-file store format of remote log metadata", it
> seems useful to specify both topic name and topic ID for debugging
> purposes.
>
> 9006. Under "Internal flat-file store format of remote log metadata", the
> description of "metadata-topic-offset" currently says "offset of the remote
> log metadata topic from which this topic partition's remote log metadata is
> fetched." Just for the wording, perhaps you meant to refer to the offset
> upto which the file has been committed? i.e. "offset of the remote log
> metadata topic upto which this topic partition's remote log metadata has
> been committed into this file."
>
> 9007. Under "Internal flat-file store format of remote log metadata", the
> schema of the payload (i.e. beyond the header) seems to contain the events
> from the metadata topic. It seems useful to instead persist the
> representation of the materialized state of the events, so that for the
> same segment only the latest state is stored. Besides reducing storage
> footprint, this also is likely to relate directly with the in-memory
> representation of the RLMM cache (which probably is some kind of a Map with
> key being segment ID and value being the segment state), so recovery from
> disk will be straightforward.
>
> 9008. Under "Topic deletion lifecycle", step (1), it will be useful to
> mention when in the deletion flow does the controller publish the
> delete_partition_marked event to say that the partition is marked for
> deletion?
>
> 9009. There are ~4 TODOs in the KIP. Could you please address these or
> remove them?
>
> 9010. There is a reference to a Google doc on the KIP which was used
> earlier for discussions. Please could you remove the reference, since the
> KIP is the source of the truth?
>
> 9011. This feedback is from an earlier comment. 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. What do you think?
>
>
> Cheers,
> Kowshik
>
>
> On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi Jun,
> > Thanks for your comments. Please go through the inline replies.
> >
> >
> > 5102.2: It seems that both positions can just be int. Another option is to
> > have two methods. Would it be clearer?
> >
> >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata,  int startPosition) throwsRemoteStorageException;
> >
> >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata, int startPosition, int endPosition) throws
> > RemoteStorageException;
> >
> > That makes sense to me, updated the KIP.
> >
> > 6003: Could you also update the javadoc for the return value?
> >
> > Updated.
> >
> > 6020: local.log.retention.bytes: Should it default to log.retention.bytes
> > to be consistent with local.log.retention.ms?
> >
> > Yes, it can be defaulted to log.retention.bytes.
> >
> > 6021: Could you define TopicIdPartition?
> >
> > Added TopicIdPartition in the KIP.
> >
> > 6022: For all public facing classes, could you specify the package name?
> >
> > Updated.
> >
> >
> > Thanks,
> > Satish.
> >
> > On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Hi, Satish,
> > >
> > > Thanks for the reply. A few more comments below.
> > >
> > > 5102.2: It seems that both positions can just be int. Another option is
> > to
> > > have two methods. Would it be clearer?
> > >
> > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > remoteLogSegmentMetadata,
> > >                                     int startPosition) throws
> > > RemoteStorageException;
> > >
> > >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > > remoteLogSegmentMetadata,
> > >                                     int startPosition, int endPosition)
> > > throws RemoteStorageException;
> > >
> > > 6003: Could you also update the javadoc for the return value?
> > >
> > > 6010: What kind of tiering throughput have you seen with 5 threads?
> > >
> > > 6020: local.log.retention.bytes: Should it default to log.retention.bytes
> > > to be consistent with local.log.retention.ms?
> > >
> > > 6021: Could you define TopicIdPartition?
> > >
> > > 6022: For all public facing classes, could you specify the package name?
> > >
> > > It seems that you already added the topicId support. Two other remaining
> > > items are (a) the format of local tier metadata storage and (b) upgrade.
> > >
> > > Jun
> > >
> > > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <sa...@gmail.com>
> > > wrote:
> > >
> > > > Hi Jun,
> > > > Thanks for your comments. Please find the inline replies below.
> > > >
> > > > >605.2 It's rare for the follower to need the remote data. So, the
> > current
> > > > approach is fine too. Could you document the process of rebuilding the
> > > > producer state since we can't simply trim the producerState to an
> > offset in
> > > > the middle of a segment.
> > > >
> > > > Will clarify in the KIP.
> > > >
> > > > >5102.2 Would it be clearer to make startPosiont long and endPosition
> > of
> > > > Optional<Long>?
> > > >
> > > > We will have arg checks with respective validation. It is not a good
> > > > practice to have arguments with optional as mentioned here.
> > > > https://rules.sonarsource.com/java/RSPEC-3553
> > > >
> > > >
> > > > >5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > > ByteBuffer.
> > > >
> > > > Updated.
> > > >
> > > > >5102.7 Could you define all public methods for LogSegmentData?
> > > >
> > > > Updated.
> > > >
> > > > >5103.5 Could you change the reference to rlm_process_interval_ms and
> > > > rlm_retry_interval_ms to the new config names? Also, the retry interval
> > > > config seems still missing. It would be useful to support exponential
> > > > backoff with the retry interval config.
> > > >
> > > > Good point. We wanted the retry with truncated exponential backoff,
> > > > updated the KIP.
> > > >
> > > > >5111. "RLM follower fetches the earliest offset for the earliest
> > leader
> > > > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> > int
> > > > leaderEpoch) and updates that as the log start offset." This text is
> > still
> > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > >
> > > > Updated.
> > > >
> > > > >5115. There are still references to "remote log cleaners".
> > > >
> > > > Updated.
> > > >
> > > > >6000. Since we are returning new error codes, we need to bump up the
> > > > protocol version for Fetch request. Also, it will be useful to
> > document all
> > > > new error codes and whether they are retriable or not.
> > > >
> > > > Sure, we will add that in the KIP.
> > > >
> > > > >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > leaderEpoch
> > > > is int32 instead of long.
> > > >
> > > > Updated.
> > > >
> > > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > > RemoteLogSegmentMetadata.state()?
> > > >
> > > > No, it is fixed.
> > > >
> > > > >6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
> > > > topicPartition, long offset, int epochForOffset): Should this return
> > > > Optional<RemoteLogSegmentMetadata>?
> > > >
> > > > That makes sense, updated.
> > > >
> > > > >6005. RemoteLogState: It seems it's better to split it between
> > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the
> > states
> > > > are never shared between the two use cases.
> > > >
> > > > Agree with that, updated.
> > > >
> > > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However,
> > is it
> > > > ture that other than the metadata topic, RLMM just needs to know
> > whether
> > > > there is a replica assigned to this broker and doesn't need to know
> > whether
> > > > the replica is the leader or the follower?
> > > >
> > > > That may be true. If the implementation does not need that, it can
> > > > ignore the information in the callback.
> > > >
> > > > >6007: "Handle expired remote segments (leader and follower)": Why is
> > this
> > > > needed in both the leader and the follower?
> > > >
> > > > Updated.
> > > >
> > > > >6008.       "name": "SegmentSizeInBytes",
> > > >                 "type": "int64",
> > > > The segment size can just be int32.
> > > >
> > > > Updated.
> > > >
> > > > >6009. For the record format in the log, it seems that we need to add
> > > > record
> > > > type and record version before the serialized bytes. We can follow the
> > > > convention used in
> > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > >
> > > > Yes, KIP already mentions that these are serialized before the payload
> > > > as below. We will mention explicitly that these two are written before
> > > > the data is written.
> > > >
> > > > RLMM instance on broker publishes the message to the topic with key as
> > > > null and value with the below format.
> > > >
> > > > type      : unsigned var int, represents the value type. This value is
> > > > 'apikey' as mentioned in the schema.
> > > > version : unsigned var int, the 'version' number of the type as
> > > > mentioned in the schema.
> > > > data      : record payload in kafka protocol message format.
> > > >
> > > >
> > > > >6010. remote.log.manager.thread.pool.size: The default value is 10.
> > This
> > > > might be too high when enabling the tiered feature for the first time.
> > > > Since there are lots of segments that need to be tiered initially, a
> > large
> > > > number of threads could overwhelm the broker.
> > > >
> > > > Is the default value 5 reasonable?
> > > >
> > > > 6011. "The number of milli seconds to keep the local log segment
> > before it
> > > > gets deleted. If not set, the value in `log.retention.minutes` is
> > used. If
> > > > set to -1, no time limit is applied." We should use log.retention.ms
> > > > instead of log.retention.minutes.
> > > > Nice typo catch. Updated the KIP.
> > > >
> > > > Thanks,
> > > > Satish.
> > > >
> > > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the updated KIP. A few more comments below.
> > > > >
> > > > > 605.2 It's rare for the follower to need the remote data. So, the
> > current
> > > > > approach is fine too. Could you document the process of rebuilding
> > the
> > > > > producer state since we can't simply trim the producerState to an
> > offset
> > > > in
> > > > > the middle of a segment.
> > > > >
> > > > > 5102.2 Would it be clearer to make startPosiont long and endPosition
> > of
> > > > > Optional<Long>?
> > > > >
> > > > > 5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > > > ByteBuffer.
> > > > >
> > > > > 5102.7 Could you define all public methods for LogSegmentData?
> > > > >
> > > > > 5103.5 Could you change the reference to rlm_process_interval_ms and
> > > > > rlm_retry_interval_ms to the new config names? Also, the retry
> > interval
> > > > > config seems still missing. It would be useful to support exponential
> > > > > backoff with the retry interval config.
> > > > >
> > > > > 5111. "RLM follower fetches the earliest offset for the earliest
> > leader
> > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > topicPartition,
> > > > int
> > > > > leaderEpoch) and updates that as the log start offset." This text is
> > > > still
> > > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > > >
> > > > > 5115. There are still references to "remote log cleaners".
> > > > >
> > > > > 6000. Since we are returning new error codes, we need to bump up the
> > > > > protocol version for Fetch request. Also, it will be useful to
> > document
> > > > all
> > > > > new error codes and whether they are retriable or not.
> > > > >
> > > > > 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > > leaderEpoch
> > > > > is int32 instead of long.
> > > > >
> > > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > > > RemoteLogSegmentMetadata.state()?
> > > > >
> > > > > 6003. RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata(TopicPartition
> > > > > topicPartition, long offset, int epochForOffset): Should this return
> > > > > Optional<RemoteLogSegmentMetadata>?
> > > > >
> > > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to pick a
> > more
> > > > > indicative name so that people understand what epoch this is.
> > > > >
> > > > > 6005. RemoteLogState: It seems it's better to split it between
> > > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the
> > states
> > > > > are never shared between the two use cases.
> > > > >
> > > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However,
> > is it
> > > > > ture that other than the metadata topic, RLMM just needs to know
> > whether
> > > > > there is a replica assigned to this broker and doesn't need to know
> > > > whether
> > > > > the replica is the leader or the follower?
> > > > >
> > > > > 6007: "Handle expired remote segments (leader and follower)": Why is
> > this
> > > > > needed in both the leader and the follower?
> > > > >
> > > > > 6008.       "name": "SegmentSizeInBytes",
> > > > >                 "type": "int64",
> > > > > The segment size can just be int32.
> > > > >
> > > > > 6009. For the record format in the log, it seems that we need to add
> > > > record
> > > > > type and record version before the serialized bytes. We can follow
> > the
> > > > > convention used in
> > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > > .
> > > > >
> > > > > 6010. remote.log.manager.thread.pool.size: The default value is 10.
> > This
> > > > > might be too high when enabling the tiered feature for the first
> > time.
> > > > > Since there are lots of segments that need to be tiered initially, a
> > > > large
> > > > > number of threads could overwhelm the broker.
> > > > >
> > > > > 6011. "The number of milli seconds to keep the local log segment
> > before
> > > > it
> > > > > gets deleted. If not set, the value in `log.retention.minutes` is
> > used.
> > > > If
> > > > > set to -1, no time limit is applied." We should use log.retention.ms
> > > > > instead of log.retention.minutes.
> > > > >
> > > > > Jun
> > > > >
> > > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> > satish.duggana@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi,
> > > > > > We updated the KIP with the points mentioned in the earlier mail
> > > > > > except for KIP-516 related changes. You can go through them and
> > let us
> > > > > > know if you have any comments. We will update the KIP with the
> > > > > > remaining todo items and KIP-516 related changes by end of this
> > > > > > week(5th Dec).
> > > > > >
> > > > > > Thanks,
> > > > > > Satish.
> > > > > >
> > > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > Hi Jun,
> > > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > > >
> > > > > > > 605.2 "Build the local leader epoch cache by cutting the leader
> > epoch
> > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > mentioned an
> > > > > > issue
> > > > > > > earlier. Suppose the leader's local start offset is 100. The
> > follower
> > > > > > finds
> > > > > > > a remote segment covering offset range [80, 120). The
> > producerState
> > > > with
> > > > > > > this remote segment is up to offset 120. To trim the
> > producerState to
> > > > > > > offset 100 requires more work since one needs to download the
> > > > previous
> > > > > > > producerState up to offset 80 and then replay the messages from
> > 80 to
> > > > > > 100.
> > > > > > > It seems that it's simpler in this case for the follower just to
> > > > take the
> > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > >
> > > > > > > We chose that approach to avoid any edge cases here. It may be
> > > > > > > possible that the remote log segment that is received may not
> > have
> > > > the
> > > > > > > same leader epoch sequence from 100-120 as it contains on the
> > > > > > > leader(this can happen due to unclean leader). It is safe to
> > start
> > > > > > > from what the leader returns here.Another way is to find the
> > remote
> > > > > > > log segment
> > > > > > >
> > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > replicas
> > > > for a
> > > > > > > partition, not on the replicas for the
> > __remote_log_segment_metadata
> > > > > > > partition. It's not clear how the leader of
> > > > __remote_log_segment_metadata
> > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > >
> > > > > > > RLMM will always receive the callback for the remote log metadata
> > > > > > > topic partitions hosted on the local broker and these will be
> > > > > > > subscribed. I will make this clear in the KIP.
> > > > > > >
> > > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> > Could
> > > > you
> > > > > > > update the KIP based on topicID?
> > > > > > >
> > > > > > > We mentioned KIP-516 and how it helps. We will update this KIP
> > with
> > > > > > > all the changes it brings with KIP-516.
> > > > > > >
> > > > > > > 5101. RLMM: It would be useful to clarify how the following two
> > APIs
> > > > are
> > > > > > > used. According to the wiki, the former is used for topic
> > deletion
> > > > and
> > > > > > the
> > > > > > > latter is used for retention. It seems that retention should use
> > the
> > > > > > former
> > > > > > > since remote segments without a matching epoch in the leader
> > > > (potentially
> > > > > > > due to unclean leader election) also need to be garbage
> > collected.
> > > > The
> > > > > > > latter seems to be used for the new leader to determine the last
> > > > tiered
> > > > > > > segment.
> > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > topicPartition, long leaderEpoch);
> > > > > > >
> > > > > > > Right,.that is what we are currently doing. We will update the
> > > > > > > javadocs and wiki with that. Earlier, we did not want to remove
> > the
> > > > > > > segments which are not matched with leader epochs from the ladder
> > > > > > > partition as they may be used later by a replica which can
> > become a
> > > > > > > leader (unclean leader election) and refer those segments. But
> > that
> > > > > > > may leak these segments in remote storage until the topic
> > lifetime.
> > > > We
> > > > > > > decided to cleanup the segments with the oldest incase of size
> > based
> > > > > > > retention also.
> > > > > > >
> > > > > > > 5102. RSM:
> > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that
> > they can
> > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > >
> > > > > > > It will be useful to have metadata for RSM to fetch log segment.
> > It
> > > > > > > may create location/path using id with other metadata too.
> > > > > > >
> > > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of
> > Long?
> > > > > > >
> > > > > > > Wanted to keep endPosition as optional to read till the end of
> > the
> > > > > > > segment and avoid sentinels.
> > > > > > >
> > > > > > > 5102.3 Why only some of the methods have default implementation
> > and
> > > > > > others
> > > > > > > Don't?
> > > > > > >
> > > > > > > Actually,  RSM will not have any default implementations. Those 3
> > > > > > > methods were made default earlier for tests etc. Updated the
> > wiki.
> > > > > > >
> > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > and DeletePartitionUpdate?
> > > > > > >
> > > > > > > Sure, they will be added.
> > > > > > >
> > > > > > >
> > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> > since
> > > > it
> > > > > > will
> > > > > > > be generated in memory.
> > > > > > >
> > > > > > > Right, this is in plan.
> > > > > > >
> > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > > baseOffset
> > > > > > and
> > > > > > > startOffset. For example, deleteRecords() could move the
> > startOffset
> > > > to
> > > > > > the
> > > > > > > middle of a segment. If we copy the full segment to remote
> > storage,
> > > > the
> > > > > > > baseOffset and the startOffset will be different.
> > > > > > >
> > > > > > > Good point. startOffset is baseOffset by default, if not set
> > > > explicitly.
> > > > > > >
> > > > > > > 5102.7 Could we define all the public methods for
> > > > > > RemoteLogSegmentMetadata
> > > > > > > and LogSegmentData?
> > > > > > >
> > > > > > > Sure, updated the wiki.
> > > > > > >
> > > > > > > 5102.8 Could we document whether endOffset in
> > > > RemoteLogSegmentMetadata is
> > > > > > > inclusive/exclusive?
> > > > > > >
> > > > > > > It is inclusive, will update.
> > > > > > >
> > > > > > > 5103. configs:
> > > > > > > 5103.1 Could we define the default value of non-required configs
> > > > (e.g the
> > > > > > > size of new thread pools)?
> > > > > > >
> > > > > > > Sure, that makes sense.
> > > > > > >
> > > > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > > > > retention.ms,
> > > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > > > >
> > > > > > > Right, we do not have  remote.log.retention as we discussed
> > earlier.
> > > > > > > Thanks for catching the typo.
> > > > > > >
> > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description says
> > > > "used in
> > > > > > > scheduling tasks to copy segments, fetch remote log indexes and
> > > > clean up
> > > > > > > remote log segments". However, there is a separate
> > > > > > > config remote.log.reader.threads for fetching remote data. It's
> > > > weird to
> > > > > > > fetch remote index and log in different thread pools since both
> > are
> > > > used
> > > > > > > for serving fetch requests.
> > > > > > >
> > > > > > > Right, remote.log.manager.thread.pool is mainly used for
> > copy/cleanup
> > > > > > > activities. Fetch path always goes through
> > remote.log.reader.threads.
> > > > > > >
> > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount
> > of
> > > > time
> > > > > > to
> > > > > > > back off when there is no work to do? If so, perhaps it can be
> > > > renamed as
> > > > > > > backoff.ms.
> > > > > > >
> > > > > > > This is the delay interval for each iteration. It may be renamed
> > to
> > > > > > > remote.log.manager.task.delay.ms
> > > > > > >
> > > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > > > configs? If
> > > > > > > so, they need to be listed in this section.
> > > > > > >
> > > > > > > remote.log.manager.task.interval.ms is the process internal,
> > retry
> > > > > > > interval is missing in the configs, which will be updated in the
> > KIP.
> > > > > > >
> > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> > > > files of
> > > > > > > remote log segments to avoid multiple index fetches from the
> > remote
> > > > > > > storage." Is the RLM in memory or on disk? If on disk, where is
> > it
> > > > > > stored?
> > > > > > > Do we need a configuration to bound the size?
> > > > > > >
> > > > > > > It is stored on disk. They are stored in a directory
> > > > > > > `remote-log-index-cache` under log dir. We plan to have a config
> > for
> > > > > > > that instead of default. We will have a configuration for that.
> > > > > > >
> > > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> > Offset
> > > > in
> > > > > > > different places. It would be useful to standardize the
> > terminology.
> > > > > > >
> > > > > > > Sure.
> > > > > > >
> > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It listed
> > two
> > > > > > options
> > > > > > > without saying which option is chosen.
> > > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > > >
> > > > > > > 5107. Follower to leader transition: It has step 2, but not step
> > 1.
> > > > > > > Step-1 is there but it is not explicitly highlighted. It is
> > previous
> > > > > > > table to step-2.
> > > > > > >
> > > > > > > 5108. If a consumer fetches from the remote data and the remote
> > > > storage
> > > > > > is
> > > > > > > not available, what error code is used in the fetch response?
> > > > > > >
> > > > > > > Good point. We have not yet defined the error for this case. We
> > need
> > > > > > > to define an error message and send the same in fetch response.
> > > > > > >
> > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> > message
> > > > > > > offset whose timestamp is >= to the given timestamp in the
> > request.
> > > > That
> > > > > > > means it checks in remote log time indexes first, after which
> > local
> > > > log
> > > > > > > time indexes are checked." Could you document which method in
> > RLMM is
> > > > > > used
> > > > > > > for this?
> > > > > > >
> > > > > > > Okay.
> > > > > > >
> > > > > > > 5110. Stopreplica: "it sets all the remote log segment metadata
> > of
> > > > that
> > > > > > > partition with a delete marker and publishes them to RLMM." This
> > > > seems
> > > > > > > outdated given the new topic deletion logic.
> > > > > > >
> > > > > > > Will update with KIP-516 related points.
> > > > > > >
> > > > > > > 5111. "RLM follower fetches the earliest offset for the earliest
> > > > leader
> > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > topicPartition,
> > > > > > int
> > > > > > > leaderEpoch) and updates that as the log start offset." Do we
> > need
> > > > that
> > > > > > > since replication propagates logStartOffset already?
> > > > > > >
> > > > > > > Good point. Right, existing replication protocol takes care of
> > > > > > > updating the followers’s log start offset received from the
> > leader.
> > > > > > >
> > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching from
> > > > remote
> > > > > > > storage?
> > > > > > >
> > > > > > > Remote reads may fail within the current default wait time, but
> > > > > > > subsequent fetches would be able to serve as that data is stored
> > in
> > > > > > > the local cache. This cache is currently implemented in RSMs.
> > But we
> > > > > > > plan to pull this into the remote log messaging layer in future.
> > > > > > >
> > > > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > > > reading
> > > > > > the
> > > > > > > messages again when a broker is restarted." Could you describe
> > the
> > > > format
> > > > > > > and the location of the file? Also, could the same message be
> > > > processed
> > > > > > by
> > > > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > > > >
> > > > > > > Sure, we will update in the KIP.
> > > > > > >
> > > > > > > 5114. Message format
> > > > > > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord
> > > > with
> > > > > > > apiKey 0 and 1.
> > > > > > >
> > > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > > >
> > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> > > > > > endOffset
> > > > > > > is inclusive/exclusive?
> > > > > > > It is inclusive, will update.
> > > > > > >
> > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > LeaderEpoch
> > > > a
> > > > > > bit
> > > > > > > more? Is that the epoch of the leader when it copies the segment
> > to
> > > > > > remote
> > > > > > > storage? Also, how will this field be used?
> > > > > > >
> > > > > > > Right, this is the leader epoch of the broker which copied this
> > > > > > > segment. This is helpful in reason about which broker copied the
> > > > > > > segment to remote storage.
> > > > > > >
> > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> > > > record in
> > > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > > >
> > > > > > > This is the  timestamp at which  the respective event occurred.
> > Added
> > > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > > > > > implementation. We thought about that but it looked cleaner to
> > use at
> > > > > > > the message structure level instead of getting that from the
> > consumer
> > > > > > > record and using that to build the respective event.
> > > > > > >
> > > > > > >
> > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > >
> > > > > > > Right, it looks like config allows only int value >= 14.
> > > > > > >
> > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the log
> > > > cleaner
> > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > RemotePartitionRemover.
> > > > > > >
> > > > > > > I am fine with RemotePartitionRemover or
> > RemoteLogDeletionManager(we
> > > > > > > have other manager classes like RLM, RLMM).
> > > > > > >
> > > > > > > 5116. "RLC receives the delete_partition_marked and processes it
> > if
> > > > it is
> > > > > > > not yet processed earlier." How does it know whether
> > > > > > > delete_partition_marked has been processed earlier?
> > > > > > >
> > > > > > > This is to handle duplicate delete_partition_marked events. RLC
> > > > > > > internally maintains a state for the delete_partition events and
> > if
> > > > it
> > > > > > > already has an existing event then it ignores if it is already
> > being
> > > > > > > processed.
> > > > > > >
> > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > metadata
> > > > > > topic?
> > > > > > >
> > > > > > > Right, this is in plan but did not mention it in the KIP. This
> > will
> > > > be
> > > > > > > useful for debugging purposes too.
> > > > > > >
> > > > > > > 5118. "Maximum remote log reader thread pool task queue size. If
> > the
> > > > task
> > > > > > > queue is full, broker will stop reading remote log segments."
> > What
> > > > do we
> > > > > > > return to the fetch request in this case?
> > > > > > >
> > > > > > > We return an error response for that partition.
> > > > > > >
> > > > > > > 5119. It would be useful to list all things not supported in the
> > > > first
> > > > > > > version in a Future work or Limitations section. For example,
> > > > compacted
> > > > > > > topic, JBOD, changing remote.log.storage.enable from true to
> > false,
> > > > etc.
> > > > > > >
> > > > > > > We already have a non-goals section which is filled with some of
> > > > these
> > > > > > > details. Do we need another limitations section?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > >
> > > > > > > > 605.2 "Build the local leader epoch cache by cutting the leader
> > > > epoch
> > > > > > > > sequence received from remote storage to [LSO, ELO]." I
> > mentioned
> > > > an
> > > > > > issue
> > > > > > > > earlier. Suppose the leader's local start offset is 100. The
> > > > follower
> > > > > > finds
> > > > > > > > a remote segment covering offset range [80, 120). The
> > producerState
> > > > > > with
> > > > > > > > this remote segment is up to offset 120. To trim the
> > producerState
> > > > to
> > > > > > > > offset 100 requires more work since one needs to download the
> > > > previous
> > > > > > > > producerState up to offset 80 and then replay the messages
> > from 80
> > > > to
> > > > > > 100.
> > > > > > > > It seems that it's simpler in this case for the follower just
> > to
> > > > take
> > > > > > the
> > > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > > >
> > > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> > replicas
> > > > for
> > > > > > a
> > > > > > > > partition, not on the replicas for the
> > > > __remote_log_segment_metadata
> > > > > > > > partition. It's not clear how the leader of
> > > > > > __remote_log_segment_metadata
> > > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > > >
> > > > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> > > > Could you
> > > > > > > > update the KIP based on topicID?
> > > > > > > >
> > > > > > > > 5101. RLMM: It would be useful to clarify how the following two
> > > > APIs
> > > > > > are
> > > > > > > > used. According to the wiki, the former is used for topic
> > deletion
> > > > and
> > > > > > the
> > > > > > > > latter is used for retention. It seems that retention should
> > use
> > > > the
> > > > > > former
> > > > > > > > since remote segments without a matching epoch in the leader
> > > > > > (potentially
> > > > > > > > due to unclean leader election) also need to be garbage
> > collected.
> > > > The
> > > > > > > > latter seems to be used for the new leader to determine the
> > last
> > > > tiered
> > > > > > > > segment.
> > > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > > listRemoteLogSegments(TopicPartition
> > > > > > > > topicPartition, long leaderEpoch);
> > > > > > > >
> > > > > > > > 5102. RSM:
> > > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that
> > they
> > > > can
> > > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of
> > > > Long?
> > > > > > > > 5102.3 Why only some of the methods have default
> > implementation and
> > > > > > others
> > > > > > > > don't?
> > > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > > and DeletePartitionUpdate?
> > > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> > > > since it
> > > > > > will
> > > > > > > > be generated in memory.
> > > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > > > > baseOffset and
> > > > > > > > startOffset. For example, deleteRecords() could move the
> > > > startOffset
> > > > > > to the
> > > > > > > > middle of a segment. If we copy the full segment to remote
> > > > storage, the
> > > > > > > > baseOffset and the startOffset will be different.
> > > > > > > > 5102.7 Could we define all the public methods for
> > > > > > RemoteLogSegmentMetadata
> > > > > > > > and LogSegmentData?
> > > > > > > > 5102.8 Could we document whether endOffset in
> > > > RemoteLogSegmentMetadata
> > > > > > is
> > > > > > > > inclusive/exclusive?
> > > > > > > >
> > > > > > > > 5103. configs:
> > > > > > > > 5103.1 Could we define the default value of non-required
> > configs
> > > > (e.g
> > > > > > the
> > > > > > > > size of new thread pools)?
> > > > > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > > > > retention.ms,
> > > > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description
> > says
> > > > "used
> > > > > > in
> > > > > > > > scheduling tasks to copy segments, fetch remote log indexes and
> > > > clean
> > > > > > up
> > > > > > > > remote log segments". However, there is a separate
> > > > > > > > config remote.log.reader.threads for fetching remote data. It's
> > > > weird
> > > > > > to
> > > > > > > > fetch remote index and log in different thread pools since
> > both are
> > > > > > used
> > > > > > > > for serving fetch requests.
> > > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> > amount of
> > > > > > time to
> > > > > > > > back off when there is no work to do? If so, perhaps it can be
> > > > renamed
> > > > > > as
> > > > > > > > backoff.ms.
> > > > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > > > configs?
> > > > > > If
> > > > > > > > so, they need to be listed in this section.
> > > > > > > >
> > > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> > > > files
> > > > > > of
> > > > > > > > remote log segments to avoid multiple index fetches from the
> > remote
> > > > > > > > storage." Is the RLM in memory or on disk? If on disk, where
> > is it
> > > > > > stored?
> > > > > > > > Do we need a configuration to bound the size?
> > > > > > > >
> > > > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> > > > Offset in
> > > > > > > > different places. It would be useful to standardize the
> > > > terminology.
> > > > > > > >
> > > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> > listed two
> > > > > > options
> > > > > > > > without saying which option is chosen.
> > > > > > > >
> > > > > > > > 5107. Follower to leader transition: It has step 2, but not
> > step 1.
> > > > > > > >
> > > > > > > > 5108. If a consumer fetches from the remote data and the remote
> > > > > > storage is
> > > > > > > > not available, what error code is used in the fetch response?
> > > > > > > >
> > > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> > > > message
> > > > > > > > offset whose timestamp is >= to the given timestamp in the
> > request.
> > > > > > That
> > > > > > > > means it checks in remote log time indexes first, after which
> > > > local log
> > > > > > > > time indexes are checked." Could you document which method in
> > RLMM
> > > > is
> > > > > > used
> > > > > > > > for this?
> > > > > > > >
> > > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> > metadata of
> > > > that
> > > > > > > > partition with a delete marker and publishes them to RLMM."
> > This
> > > > seems
> > > > > > > > outdated given the new topic deletion logic.
> > > > > > > >
> > > > > > > > 5111. "RLM follower fetches the earliest offset for the
> > earliest
> > > > leader
> > > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > > topicPartition,
> > > > > > int
> > > > > > > > leaderEpoch) and updates that as the log start offset." Do we
> > need
> > > > that
> > > > > > > > since replication propagates logStartOffset already?
> > > > > > > >
> > > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching
> > from
> > > > remote
> > > > > > > > storage?
> > > > > > > >
> > > > > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > > > > > reading the
> > > > > > > > messages again when a broker is restarted." Could you describe
> > the
> > > > > > format
> > > > > > > > and the location of the file? Also, could the same message be
> > > > > > processed by
> > > > > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > > > > >
> > > > > > > > 5114. Message format
> > > > > > > > 5114.1 There are two records named
> > RemoteLogSegmentMetadataRecord
> > > > with
> > > > > > > > apiKey 0 and 1.
> > > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> > whether
> > > > > > endOffset
> > > > > > > > is inclusive/exclusive?
> > > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > > LeaderEpoch a
> > > > > > bit
> > > > > > > > more? Is that the epoch of the leader when it copies the
> > segment to
> > > > > > remote
> > > > > > > > storage? Also, how will this field be used?
> > > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> > > > record
> > > > > > in
> > > > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > > >
> > > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the
> > log
> > > > > > cleaner
> > > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > > RemotePartitionRemover.
> > > > > > > >
> > > > > > > > 5116. "RLC receives the delete_partition_marked and processes
> > it
> > > > if it
> > > > > > is
> > > > > > > > not yet processed earlier." How does it know whether
> > > > > > > > delete_partition_marked has been processed earlier?
> > > > > > > >
> > > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > > > metadata
> > > > > > topic?
> > > > > > > >
> > > > > > > > 5118. "Maximum remote log reader thread pool task queue size.
> > If
> > > > the
> > > > > > task
> > > > > > > > queue is full, broker will stop reading remote log segments."
> > What
> > > > do
> > > > > > we
> > > > > > > > return to the fetch request in this case?
> > > > > > > >
> > > > > > > > 5119. It would be useful to list all things not supported in
> > the
> > > > first
> > > > > > > > version in a Future work or Limitations section. For example,
> > > > compacted
> > > > > > > > topic, JBOD, changing remote.log.storage.enable from true to
> > false,
> > > > > > etc.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > > (
> > > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> > kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > > >>>>>>>
> > > > > > > > > >
> > > > > >
> > > > > >
> > > >
> >


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

Posted by Kowshik Prakasam <kp...@confluent.io>.
Hi Satish,

Thanks for the updates! A few more comments below.

9001. Under the "Upgrade" section, there is a line mentioning: "Upgrade the
existing Kafka cluster to 2.7 version and allow this to run for the log
retention of user topics that you want to enable tiered storage. This will
allow all the topics to have the producer snapshots generated for each log
segment." -- Which associated change in AK were you referring to here? Is
it: https://github.com/apache/kafka/pull/7929 ? It seems like I don't see
it in the 2.7 release branch yet, here is the link:
https://github.com/apache/kafka/commits/2.7.

9002. Under the "Upgrade" section, the configuration mentioned is
'remote.log.storage.system.enable'. However, under "Public Interfaces"
section the corresponding configuration is 'remote.storage.system.enable'.
Could we use the same one in both, maybe 'remote.log.storage.system.enable'?

9003. Under "Per Topic Configuration", the KIP recommends setting
'remote.log.storage.enable' to true at a per-topic level. It will be useful
to add a line that if the user wants to enable it for all topics, then they
should be able to set the cluster-wide default to true. Also, it will be
useful to mention that the KIP currently does not support setting it to
false (after it is set to true), and add that to the future work section.

9004. Under "Committed offsets file format", the sample provided shows
partition number and offset. Is the topic name required for identifying
which topic the partitions belong to?

9005. Under "Internal flat-file store format of remote log metadata", it
seems useful to specify both topic name and topic ID for debugging
purposes.

9006. Under "Internal flat-file store format of remote log metadata", the
description of "metadata-topic-offset" currently says "offset of the remote
log metadata topic from which this topic partition's remote log metadata is
fetched." Just for the wording, perhaps you meant to refer to the offset
upto which the file has been committed? i.e. "offset of the remote log
metadata topic upto which this topic partition's remote log metadata has
been committed into this file."

9007. Under "Internal flat-file store format of remote log metadata", the
schema of the payload (i.e. beyond the header) seems to contain the events
from the metadata topic. It seems useful to instead persist the
representation of the materialized state of the events, so that for the
same segment only the latest state is stored. Besides reducing storage
footprint, this also is likely to relate directly with the in-memory
representation of the RLMM cache (which probably is some kind of a Map with
key being segment ID and value being the segment state), so recovery from
disk will be straightforward.

9008. Under "Topic deletion lifecycle", step (1), it will be useful to
mention when in the deletion flow does the controller publish the
delete_partition_marked event to say that the partition is marked for
deletion?

9009. There are ~4 TODOs in the KIP. Could you please address these or
remove them?

9010. There is a reference to a Google doc on the KIP which was used
earlier for discussions. Please could you remove the reference, since the
KIP is the source of the truth?

9011. This feedback is from an earlier comment. 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. What do you think?


Cheers,
Kowshik


On Mon, Dec 14, 2020 at 11:07 AM Satish Duggana <sa...@gmail.com>
wrote:

> Hi Jun,
> Thanks for your comments. Please go through the inline replies.
>
>
> 5102.2: It seems that both positions can just be int. Another option is to
> have two methods. Would it be clearer?
>
>     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> remoteLogSegmentMetadata,  int startPosition) throwsRemoteStorageException;
>
>     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> remoteLogSegmentMetadata, int startPosition, int endPosition) throws
> RemoteStorageException;
>
> That makes sense to me, updated the KIP.
>
> 6003: Could you also update the javadoc for the return value?
>
> Updated.
>
> 6020: local.log.retention.bytes: Should it default to log.retention.bytes
> to be consistent with local.log.retention.ms?
>
> Yes, it can be defaulted to log.retention.bytes.
>
> 6021: Could you define TopicIdPartition?
>
> Added TopicIdPartition in the KIP.
>
> 6022: For all public facing classes, could you specify the package name?
>
> Updated.
>
>
> Thanks,
> Satish.
>
> On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io> wrote:
> >
> > Hi, Satish,
> >
> > Thanks for the reply. A few more comments below.
> >
> > 5102.2: It seems that both positions can just be int. Another option is
> to
> > have two methods. Would it be clearer?
> >
> >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata,
> >                                     int startPosition) throws
> > RemoteStorageException;
> >
> >     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata,
> >                                     int startPosition, int endPosition)
> > throws RemoteStorageException;
> >
> > 6003: Could you also update the javadoc for the return value?
> >
> > 6010: What kind of tiering throughput have you seen with 5 threads?
> >
> > 6020: local.log.retention.bytes: Should it default to log.retention.bytes
> > to be consistent with local.log.retention.ms?
> >
> > 6021: Could you define TopicIdPartition?
> >
> > 6022: For all public facing classes, could you specify the package name?
> >
> > It seems that you already added the topicId support. Two other remaining
> > items are (a) the format of local tier metadata storage and (b) upgrade.
> >
> > Jun
> >
> > On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <sa...@gmail.com>
> > wrote:
> >
> > > Hi Jun,
> > > Thanks for your comments. Please find the inline replies below.
> > >
> > > >605.2 It's rare for the follower to need the remote data. So, the
> current
> > > approach is fine too. Could you document the process of rebuilding the
> > > producer state since we can't simply trim the producerState to an
> offset in
> > > the middle of a segment.
> > >
> > > Will clarify in the KIP.
> > >
> > > >5102.2 Would it be clearer to make startPosiont long and endPosition
> of
> > > Optional<Long>?
> > >
> > > We will have arg checks with respective validation. It is not a good
> > > practice to have arguments with optional as mentioned here.
> > > https://rules.sonarsource.com/java/RSPEC-3553
> > >
> > >
> > > >5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > ByteBuffer.
> > >
> > > Updated.
> > >
> > > >5102.7 Could you define all public methods for LogSegmentData?
> > >
> > > Updated.
> > >
> > > >5103.5 Could you change the reference to rlm_process_interval_ms and
> > > rlm_retry_interval_ms to the new config names? Also, the retry interval
> > > config seems still missing. It would be useful to support exponential
> > > backoff with the retry interval config.
> > >
> > > Good point. We wanted the retry with truncated exponential backoff,
> > > updated the KIP.
> > >
> > > >5111. "RLM follower fetches the earliest offset for the earliest
> leader
> > > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> int
> > > leaderEpoch) and updates that as the log start offset." This text is
> still
> > > there. Also, could we remove earliestLogOffset() from RLMM?
> > >
> > > Updated.
> > >
> > > >5115. There are still references to "remote log cleaners".
> > >
> > > Updated.
> > >
> > > >6000. Since we are returning new error codes, we need to bump up the
> > > protocol version for Fetch request. Also, it will be useful to
> document all
> > > new error codes and whether they are retriable or not.
> > >
> > > Sure, we will add that in the KIP.
> > >
> > > >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> leaderEpoch
> > > is int32 instead of long.
> > >
> > > Updated.
> > >
> > > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > RemoteLogSegmentMetadata.state()?
> > >
> > > No, it is fixed.
> > >
> > > >6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
> > > topicPartition, long offset, int epochForOffset): Should this return
> > > Optional<RemoteLogSegmentMetadata>?
> > >
> > > That makes sense, updated.
> > >
> > > >6005. RemoteLogState: It seems it's better to split it between
> > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the
> states
> > > are never shared between the two use cases.
> > >
> > > Agree with that, updated.
> > >
> > > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However,
> is it
> > > ture that other than the metadata topic, RLMM just needs to know
> whether
> > > there is a replica assigned to this broker and doesn't need to know
> whether
> > > the replica is the leader or the follower?
> > >
> > > That may be true. If the implementation does not need that, it can
> > > ignore the information in the callback.
> > >
> > > >6007: "Handle expired remote segments (leader and follower)": Why is
> this
> > > needed in both the leader and the follower?
> > >
> > > Updated.
> > >
> > > >6008.       "name": "SegmentSizeInBytes",
> > >                 "type": "int64",
> > > The segment size can just be int32.
> > >
> > > Updated.
> > >
> > > >6009. For the record format in the log, it seems that we need to add
> > > record
> > > type and record version before the serialized bytes. We can follow the
> > > convention used in
> > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > >
> > > Yes, KIP already mentions that these are serialized before the payload
> > > as below. We will mention explicitly that these two are written before
> > > the data is written.
> > >
> > > RLMM instance on broker publishes the message to the topic with key as
> > > null and value with the below format.
> > >
> > > type      : unsigned var int, represents the value type. This value is
> > > 'apikey' as mentioned in the schema.
> > > version : unsigned var int, the 'version' number of the type as
> > > mentioned in the schema.
> > > data      : record payload in kafka protocol message format.
> > >
> > >
> > > >6010. remote.log.manager.thread.pool.size: The default value is 10.
> This
> > > might be too high when enabling the tiered feature for the first time.
> > > Since there are lots of segments that need to be tiered initially, a
> large
> > > number of threads could overwhelm the broker.
> > >
> > > Is the default value 5 reasonable?
> > >
> > > 6011. "The number of milli seconds to keep the local log segment
> before it
> > > gets deleted. If not set, the value in `log.retention.minutes` is
> used. If
> > > set to -1, no time limit is applied." We should use log.retention.ms
> > > instead of log.retention.minutes.
> > > Nice typo catch. Updated the KIP.
> > >
> > > Thanks,
> > > Satish.
> > >
> > > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > Hi, Satish,
> > > >
> > > > Thanks for the updated KIP. A few more comments below.
> > > >
> > > > 605.2 It's rare for the follower to need the remote data. So, the
> current
> > > > approach is fine too. Could you document the process of rebuilding
> the
> > > > producer state since we can't simply trim the producerState to an
> offset
> > > in
> > > > the middle of a segment.
> > > >
> > > > 5102.2 Would it be clearer to make startPosiont long and endPosition
> of
> > > > Optional<Long>?
> > > >
> > > > 5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > > ByteBuffer.
> > > >
> > > > 5102.7 Could you define all public methods for LogSegmentData?
> > > >
> > > > 5103.5 Could you change the reference to rlm_process_interval_ms and
> > > > rlm_retry_interval_ms to the new config names? Also, the retry
> interval
> > > > config seems still missing. It would be useful to support exponential
> > > > backoff with the retry interval config.
> > > >
> > > > 5111. "RLM follower fetches the earliest offset for the earliest
> leader
> > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> topicPartition,
> > > int
> > > > leaderEpoch) and updates that as the log start offset." This text is
> > > still
> > > > there. Also, could we remove earliestLogOffset() from RLMM?
> > > >
> > > > 5115. There are still references to "remote log cleaners".
> > > >
> > > > 6000. Since we are returning new error codes, we need to bump up the
> > > > protocol version for Fetch request. Also, it will be useful to
> document
> > > all
> > > > new error codes and whether they are retriable or not.
> > > >
> > > > 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > > leaderEpoch
> > > > is int32 instead of long.
> > > >
> > > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > > RemoteLogSegmentMetadata.state()?
> > > >
> > > > 6003. RemoteLogSegmentMetadata
> remoteLogSegmentMetadata(TopicPartition
> > > > topicPartition, long offset, int epochForOffset): Should this return
> > > > Optional<RemoteLogSegmentMetadata>?
> > > >
> > > > 6004. DeletePartitionUpdate.epoch(): It would be useful to pick a
> more
> > > > indicative name so that people understand what epoch this is.
> > > >
> > > > 6005. RemoteLogState: It seems it's better to split it between
> > > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the
> states
> > > > are never shared between the two use cases.
> > > >
> > > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However,
> is it
> > > > ture that other than the metadata topic, RLMM just needs to know
> whether
> > > > there is a replica assigned to this broker and doesn't need to know
> > > whether
> > > > the replica is the leader or the follower?
> > > >
> > > > 6007: "Handle expired remote segments (leader and follower)": Why is
> this
> > > > needed in both the leader and the follower?
> > > >
> > > > 6008.       "name": "SegmentSizeInBytes",
> > > >                 "type": "int64",
> > > > The segment size can just be int32.
> > > >
> > > > 6009. For the record format in the log, it seems that we need to add
> > > record
> > > > type and record version before the serialized bytes. We can follow
> the
> > > > convention used in
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > > .
> > > >
> > > > 6010. remote.log.manager.thread.pool.size: The default value is 10.
> This
> > > > might be too high when enabling the tiered feature for the first
> time.
> > > > Since there are lots of segments that need to be tiered initially, a
> > > large
> > > > number of threads could overwhelm the broker.
> > > >
> > > > 6011. "The number of milli seconds to keep the local log segment
> before
> > > it
> > > > gets deleted. If not set, the value in `log.retention.minutes` is
> used.
> > > If
> > > > set to -1, no time limit is applied." We should use log.retention.ms
> > > > instead of log.retention.minutes.
> > > >
> > > > Jun
> > > >
> > > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <
> satish.duggana@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi,
> > > > > We updated the KIP with the points mentioned in the earlier mail
> > > > > except for KIP-516 related changes. You can go through them and
> let us
> > > > > know if you have any comments. We will update the KIP with the
> > > > > remaining todo items and KIP-516 related changes by end of this
> > > > > week(5th Dec).
> > > > >
> > > > > Thanks,
> > > > > Satish.
> > > > >
> > > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > > satish.duggana@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > Hi Jun,
> > > > > > Thanks for your comments. Please find the inline replies below.
> > > > > >
> > > > > > 605.2 "Build the local leader epoch cache by cutting the leader
> epoch
> > > > > > sequence received from remote storage to [LSO, ELO]." I
> mentioned an
> > > > > issue
> > > > > > earlier. Suppose the leader's local start offset is 100. The
> follower
> > > > > finds
> > > > > > a remote segment covering offset range [80, 120). The
> producerState
> > > with
> > > > > > this remote segment is up to offset 120. To trim the
> producerState to
> > > > > > offset 100 requires more work since one needs to download the
> > > previous
> > > > > > producerState up to offset 80 and then replay the messages from
> 80 to
> > > > > 100.
> > > > > > It seems that it's simpler in this case for the follower just to
> > > take the
> > > > > > remote segment as it is and start fetching from offset 120.
> > > > > >
> > > > > > We chose that approach to avoid any edge cases here. It may be
> > > > > > possible that the remote log segment that is received may not
> have
> > > the
> > > > > > same leader epoch sequence from 100-120 as it contains on the
> > > > > > leader(this can happen due to unclean leader). It is safe to
> start
> > > > > > from what the leader returns here.Another way is to find the
> remote
> > > > > > log segment
> > > > > >
> > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> replicas
> > > for a
> > > > > > partition, not on the replicas for the
> __remote_log_segment_metadata
> > > > > > partition. It's not clear how the leader of
> > > __remote_log_segment_metadata
> > > > > > obtains the metadata for remote segments for deletion.
> > > > > >
> > > > > > RLMM will always receive the callback for the remote log metadata
> > > > > > topic partitions hosted on the local broker and these will be
> > > > > > subscribed. I will make this clear in the KIP.
> > > > > >
> > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> Could
> > > you
> > > > > > update the KIP based on topicID?
> > > > > >
> > > > > > We mentioned KIP-516 and how it helps. We will update this KIP
> with
> > > > > > all the changes it brings with KIP-516.
> > > > > >
> > > > > > 5101. RLMM: It would be useful to clarify how the following two
> APIs
> > > are
> > > > > > used. According to the wiki, the former is used for topic
> deletion
> > > and
> > > > > the
> > > > > > latter is used for retention. It seems that retention should use
> the
> > > > > former
> > > > > > since remote segments without a matching epoch in the leader
> > > (potentially
> > > > > > due to unclean leader election) also need to be garbage
> collected.
> > > The
> > > > > > latter seems to be used for the new leader to determine the last
> > > tiered
> > > > > > segment.
> > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > listRemoteLogSegments(TopicPartition
> > > > > > topicPartition, long leaderEpoch);
> > > > > >
> > > > > > Right,.that is what we are currently doing. We will update the
> > > > > > javadocs and wiki with that. Earlier, we did not want to remove
> the
> > > > > > segments which are not matched with leader epochs from the ladder
> > > > > > partition as they may be used later by a replica which can
> become a
> > > > > > leader (unclean leader election) and refer those segments. But
> that
> > > > > > may leak these segments in remote storage until the topic
> lifetime.
> > > We
> > > > > > decided to cleanup the segments with the oldest incase of size
> based
> > > > > > retention also.
> > > > > >
> > > > > > 5102. RSM:
> > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that
> they can
> > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > >
> > > > > > It will be useful to have metadata for RSM to fetch log segment.
> It
> > > > > > may create location/path using id with other metadata too.
> > > > > >
> > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of
> Long?
> > > > > >
> > > > > > Wanted to keep endPosition as optional to read till the end of
> the
> > > > > > segment and avoid sentinels.
> > > > > >
> > > > > > 5102.3 Why only some of the methods have default implementation
> and
> > > > > others
> > > > > > Don't?
> > > > > >
> > > > > > Actually,  RSM will not have any default implementations. Those 3
> > > > > > methods were made default earlier for tests etc. Updated the
> wiki.
> > > > > >
> > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > and DeletePartitionUpdate?
> > > > > >
> > > > > > Sure, they will be added.
> > > > > >
> > > > > >
> > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> since
> > > it
> > > > > will
> > > > > > be generated in memory.
> > > > > >
> > > > > > Right, this is in plan.
> > > > > >
> > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > baseOffset
> > > > > and
> > > > > > startOffset. For example, deleteRecords() could move the
> startOffset
> > > to
> > > > > the
> > > > > > middle of a segment. If we copy the full segment to remote
> storage,
> > > the
> > > > > > baseOffset and the startOffset will be different.
> > > > > >
> > > > > > Good point. startOffset is baseOffset by default, if not set
> > > explicitly.
> > > > > >
> > > > > > 5102.7 Could we define all the public methods for
> > > > > RemoteLogSegmentMetadata
> > > > > > and LogSegmentData?
> > > > > >
> > > > > > Sure, updated the wiki.
> > > > > >
> > > > > > 5102.8 Could we document whether endOffset in
> > > RemoteLogSegmentMetadata is
> > > > > > inclusive/exclusive?
> > > > > >
> > > > > > It is inclusive, will update.
> > > > > >
> > > > > > 5103. configs:
> > > > > > 5103.1 Could we define the default value of non-required configs
> > > (e.g the
> > > > > > size of new thread pools)?
> > > > > >
> > > > > > Sure, that makes sense.
> > > > > >
> > > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > > > retention.ms,
> > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > > >
> > > > > > Right, we do not have  remote.log.retention as we discussed
> earlier.
> > > > > > Thanks for catching the typo.
> > > > > >
> > > > > > 5103.3 remote.log.manager.thread.pool.size: The description says
> > > "used in
> > > > > > scheduling tasks to copy segments, fetch remote log indexes and
> > > clean up
> > > > > > remote log segments". However, there is a separate
> > > > > > config remote.log.reader.threads for fetching remote data. It's
> > > weird to
> > > > > > fetch remote index and log in different thread pools since both
> are
> > > used
> > > > > > for serving fetch requests.
> > > > > >
> > > > > > Right, remote.log.manager.thread.pool is mainly used for
> copy/cleanup
> > > > > > activities. Fetch path always goes through
> remote.log.reader.threads.
> > > > > >
> > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount
> of
> > > time
> > > > > to
> > > > > > back off when there is no work to do? If so, perhaps it can be
> > > renamed as
> > > > > > backoff.ms.
> > > > > >
> > > > > > This is the delay interval for each iteration. It may be renamed
> to
> > > > > > remote.log.manager.task.delay.ms
> > > > > >
> > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > > configs? If
> > > > > > so, they need to be listed in this section.
> > > > > >
> > > > > > remote.log.manager.task.interval.ms is the process internal,
> retry
> > > > > > interval is missing in the configs, which will be updated in the
> KIP.
> > > > > >
> > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> > > files of
> > > > > > remote log segments to avoid multiple index fetches from the
> remote
> > > > > > storage." Is the RLM in memory or on disk? If on disk, where is
> it
> > > > > stored?
> > > > > > Do we need a configuration to bound the size?
> > > > > >
> > > > > > It is stored on disk. They are stored in a directory
> > > > > > `remote-log-index-cache` under log dir. We plan to have a config
> for
> > > > > > that instead of default. We will have a configuration for that.
> > > > > >
> > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> Offset
> > > in
> > > > > > different places. It would be useful to standardize the
> terminology.
> > > > > >
> > > > > > Sure.
> > > > > >
> > > > > > 5106. The section on "In BuildingRemoteLogAux state". It listed
> two
> > > > > options
> > > > > > without saying which option is chosen.
> > > > > > We already mentioned in the KIP that we chose option-2.
> > > > > >
> > > > > > 5107. Follower to leader transition: It has step 2, but not step
> 1.
> > > > > > Step-1 is there but it is not explicitly highlighted. It is
> previous
> > > > > > table to step-2.
> > > > > >
> > > > > > 5108. If a consumer fetches from the remote data and the remote
> > > storage
> > > > > is
> > > > > > not available, what error code is used in the fetch response?
> > > > > >
> > > > > > Good point. We have not yet defined the error for this case. We
> need
> > > > > > to define an error message and send the same in fetch response.
> > > > > >
> > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> message
> > > > > > offset whose timestamp is >= to the given timestamp in the
> request.
> > > That
> > > > > > means it checks in remote log time indexes first, after which
> local
> > > log
> > > > > > time indexes are checked." Could you document which method in
> RLMM is
> > > > > used
> > > > > > for this?
> > > > > >
> > > > > > Okay.
> > > > > >
> > > > > > 5110. Stopreplica: "it sets all the remote log segment metadata
> of
> > > that
> > > > > > partition with a delete marker and publishes them to RLMM." This
> > > seems
> > > > > > outdated given the new topic deletion logic.
> > > > > >
> > > > > > Will update with KIP-516 related points.
> > > > > >
> > > > > > 5111. "RLM follower fetches the earliest offset for the earliest
> > > leader
> > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > topicPartition,
> > > > > int
> > > > > > leaderEpoch) and updates that as the log start offset." Do we
> need
> > > that
> > > > > > since replication propagates logStartOffset already?
> > > > > >
> > > > > > Good point. Right, existing replication protocol takes care of
> > > > > > updating the followers’s log start offset received from the
> leader.
> > > > > >
> > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching from
> > > remote
> > > > > > storage?
> > > > > >
> > > > > > Remote reads may fail within the current default wait time, but
> > > > > > subsequent fetches would be able to serve as that data is stored
> in
> > > > > > the local cache. This cache is currently implemented in RSMs.
> But we
> > > > > > plan to pull this into the remote log messaging layer in future.
> > > > > >
> > > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > > reading
> > > > > the
> > > > > > messages again when a broker is restarted." Could you describe
> the
> > > format
> > > > > > and the location of the file? Also, could the same message be
> > > processed
> > > > > by
> > > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > > >
> > > > > > Sure, we will update in the KIP.
> > > > > >
> > > > > > 5114. Message format
> > > > > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord
> > > with
> > > > > > apiKey 0 and 1.
> > > > > >
> > > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > > >
> > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> > > > > endOffset
> > > > > > is inclusive/exclusive?
> > > > > > It is inclusive, will update.
> > > > > >
> > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> LeaderEpoch
> > > a
> > > > > bit
> > > > > > more? Is that the epoch of the leader when it copies the segment
> to
> > > > > remote
> > > > > > storage? Also, how will this field be used?
> > > > > >
> > > > > > Right, this is the leader epoch of the broker which copied this
> > > > > > segment. This is helpful in reason about which broker copied the
> > > > > > segment to remote storage.
> > > > > >
> > > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> > > record in
> > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > >
> > > > > > This is the  timestamp at which  the respective event occurred.
> Added
> > > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > > > > implementation. We thought about that but it looked cleaner to
> use at
> > > > > > the message structure level instead of getting that from the
> consumer
> > > > > > record and using that to build the respective event.
> > > > > >
> > > > > >
> > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > >
> > > > > > Right, it looks like config allows only int value >= 14.
> > > > > >
> > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the log
> > > cleaner
> > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > RemotePartitionRemover.
> > > > > >
> > > > > > I am fine with RemotePartitionRemover or
> RemoteLogDeletionManager(we
> > > > > > have other manager classes like RLM, RLMM).
> > > > > >
> > > > > > 5116. "RLC receives the delete_partition_marked and processes it
> if
> > > it is
> > > > > > not yet processed earlier." How does it know whether
> > > > > > delete_partition_marked has been processed earlier?
> > > > > >
> > > > > > This is to handle duplicate delete_partition_marked events. RLC
> > > > > > internally maintains a state for the delete_partition events and
> if
> > > it
> > > > > > already has an existing event then it ignores if it is already
> being
> > > > > > processed.
> > > > > >
> > > > > > 5117. Should we add a new MessageFormatter to read the tier
> metadata
> > > > > topic?
> > > > > >
> > > > > > Right, this is in plan but did not mention it in the KIP. This
> will
> > > be
> > > > > > useful for debugging purposes too.
> > > > > >
> > > > > > 5118. "Maximum remote log reader thread pool task queue size. If
> the
> > > task
> > > > > > queue is full, broker will stop reading remote log segments."
> What
> > > do we
> > > > > > return to the fetch request in this case?
> > > > > >
> > > > > > We return an error response for that partition.
> > > > > >
> > > > > > 5119. It would be useful to list all things not supported in the
> > > first
> > > > > > version in a Future work or Limitations section. For example,
> > > compacted
> > > > > > topic, JBOD, changing remote.log.storage.enable from true to
> false,
> > > etc.
> > > > > >
> > > > > > We already have a non-goals section which is filled with some of
> > > these
> > > > > > details. Do we need another limitations section?
> > > > > >
> > > > > > Thanks,
> > > > > > Satish.
> > > > > >
> > > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > >
> > > > > > > Hi, Satish,
> > > > > > >
> > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > >
> > > > > > > 605.2 "Build the local leader epoch cache by cutting the leader
> > > epoch
> > > > > > > sequence received from remote storage to [LSO, ELO]." I
> mentioned
> > > an
> > > > > issue
> > > > > > > earlier. Suppose the leader's local start offset is 100. The
> > > follower
> > > > > finds
> > > > > > > a remote segment covering offset range [80, 120). The
> producerState
> > > > > with
> > > > > > > this remote segment is up to offset 120. To trim the
> producerState
> > > to
> > > > > > > offset 100 requires more work since one needs to download the
> > > previous
> > > > > > > producerState up to offset 80 and then replay the messages
> from 80
> > > to
> > > > > 100.
> > > > > > > It seems that it's simpler in this case for the follower just
> to
> > > take
> > > > > the
> > > > > > > remote segment as it is and start fetching from offset 120.
> > > > > > >
> > > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > > RLMM.onPartitionLeadershipChanges() is only called on the
> replicas
> > > for
> > > > > a
> > > > > > > partition, not on the replicas for the
> > > __remote_log_segment_metadata
> > > > > > > partition. It's not clear how the leader of
> > > > > __remote_log_segment_metadata
> > > > > > > obtains the metadata for remote segments for deletion.
> > > > > > >
> > > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> > > Could you
> > > > > > > update the KIP based on topicID?
> > > > > > >
> > > > > > > 5101. RLMM: It would be useful to clarify how the following two
> > > APIs
> > > > > are
> > > > > > > used. According to the wiki, the former is used for topic
> deletion
> > > and
> > > > > the
> > > > > > > latter is used for retention. It seems that retention should
> use
> > > the
> > > > > former
> > > > > > > since remote segments without a matching epoch in the leader
> > > > > (potentially
> > > > > > > due to unclean leader election) also need to be garbage
> collected.
> > > The
> > > > > > > latter seems to be used for the new leader to determine the
> last
> > > tiered
> > > > > > > segment.
> > > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > > listRemoteLogSegments(TopicPartition
> > > > > > > topicPartition, long leaderEpoch);
> > > > > > >
> > > > > > > 5102. RSM:
> > > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that
> they
> > > can
> > > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of
> > > Long?
> > > > > > > 5102.3 Why only some of the methods have default
> implementation and
> > > > > others
> > > > > > > don't?
> > > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > > and DeletePartitionUpdate?
> > > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> > > since it
> > > > > will
> > > > > > > be generated in memory.
> > > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > > > baseOffset and
> > > > > > > startOffset. For example, deleteRecords() could move the
> > > startOffset
> > > > > to the
> > > > > > > middle of a segment. If we copy the full segment to remote
> > > storage, the
> > > > > > > baseOffset and the startOffset will be different.
> > > > > > > 5102.7 Could we define all the public methods for
> > > > > RemoteLogSegmentMetadata
> > > > > > > and LogSegmentData?
> > > > > > > 5102.8 Could we document whether endOffset in
> > > RemoteLogSegmentMetadata
> > > > > is
> > > > > > > inclusive/exclusive?
> > > > > > >
> > > > > > > 5103. configs:
> > > > > > > 5103.1 Could we define the default value of non-required
> configs
> > > (e.g
> > > > > the
> > > > > > > size of new thread pools)?
> > > > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > > > retention.ms,
> > > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > > > > 5103.3 remote.log.manager.thread.pool.size: The description
> says
> > > "used
> > > > > in
> > > > > > > scheduling tasks to copy segments, fetch remote log indexes and
> > > clean
> > > > > up
> > > > > > > remote log segments". However, there is a separate
> > > > > > > config remote.log.reader.threads for fetching remote data. It's
> > > weird
> > > > > to
> > > > > > > fetch remote index and log in different thread pools since
> both are
> > > > > used
> > > > > > > for serving fetch requests.
> > > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the
> amount of
> > > > > time to
> > > > > > > back off when there is no work to do? If so, perhaps it can be
> > > renamed
> > > > > as
> > > > > > > backoff.ms.
> > > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > > configs?
> > > > > If
> > > > > > > so, they need to be listed in this section.
> > > > > > >
> > > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> > > files
> > > > > of
> > > > > > > remote log segments to avoid multiple index fetches from the
> remote
> > > > > > > storage." Is the RLM in memory or on disk? If on disk, where
> is it
> > > > > stored?
> > > > > > > Do we need a configuration to bound the size?
> > > > > > >
> > > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> > > Offset in
> > > > > > > different places. It would be useful to standardize the
> > > terminology.
> > > > > > >
> > > > > > > 5106. The section on "In BuildingRemoteLogAux state". It
> listed two
> > > > > options
> > > > > > > without saying which option is chosen.
> > > > > > >
> > > > > > > 5107. Follower to leader transition: It has step 2, but not
> step 1.
> > > > > > >
> > > > > > > 5108. If a consumer fetches from the remote data and the remote
> > > > > storage is
> > > > > > > not available, what error code is used in the fetch response?
> > > > > > >
> > > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> > > message
> > > > > > > offset whose timestamp is >= to the given timestamp in the
> request.
> > > > > That
> > > > > > > means it checks in remote log time indexes first, after which
> > > local log
> > > > > > > time indexes are checked." Could you document which method in
> RLMM
> > > is
> > > > > used
> > > > > > > for this?
> > > > > > >
> > > > > > > 5110. Stopreplica: "it sets all the remote log segment
> metadata of
> > > that
> > > > > > > partition with a delete marker and publishes them to RLMM."
> This
> > > seems
> > > > > > > outdated given the new topic deletion logic.
> > > > > > >
> > > > > > > 5111. "RLM follower fetches the earliest offset for the
> earliest
> > > leader
> > > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > > topicPartition,
> > > > > int
> > > > > > > leaderEpoch) and updates that as the log start offset." Do we
> need
> > > that
> > > > > > > since replication propagates logStartOffset already?
> > > > > > >
> > > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching
> from
> > > remote
> > > > > > > storage?
> > > > > > >
> > > > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > > > > reading the
> > > > > > > messages again when a broker is restarted." Could you describe
> the
> > > > > format
> > > > > > > and the location of the file? Also, could the same message be
> > > > > processed by
> > > > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > > > >
> > > > > > > 5114. Message format
> > > > > > > 5114.1 There are two records named
> RemoteLogSegmentMetadataRecord
> > > with
> > > > > > > apiKey 0 and 1.
> > > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document
> whether
> > > > > endOffset
> > > > > > > is inclusive/exclusive?
> > > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > > LeaderEpoch a
> > > > > bit
> > > > > > > more? Is that the epoch of the leader when it copies the
> segment to
> > > > > remote
> > > > > > > storage? Also, how will this field be used?
> > > > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> > > record
> > > > > in
> > > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > > >
> > > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the
> log
> > > > > cleaner
> > > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > > RemotePartitionRemover.
> > > > > > >
> > > > > > > 5116. "RLC receives the delete_partition_marked and processes
> it
> > > if it
> > > > > is
> > > > > > > not yet processed earlier." How does it know whether
> > > > > > > delete_partition_marked has been processed earlier?
> > > > > > >
> > > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > > metadata
> > > > > topic?
> > > > > > >
> > > > > > > 5118. "Maximum remote log reader thread pool task queue size.
> If
> > > the
> > > > > task
> > > > > > > queue is full, broker will stop reading remote log segments."
> What
> > > do
> > > > > we
> > > > > > > return to the fetch request in this case?
> > > > > > >
> > > > > > > 5119. It would be useful to list all things not supported in
> the
> > > first
> > > > > > > version in a Future work or Limitations section. For example,
> > > compacted
> > > > > > > topic, JBOD, changing remote.log.storage.enable from true to
> false,
> > > > > etc.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > > kprakasam@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.duggana@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.duggana@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 <
> > > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > > (
> > > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > > >>>>>> yingz@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.ch@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 (
> kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > > >>>>>>>
> > > > > > > > >
> > > > >
> > > > >
> > >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Jun,
Thanks for your comments. Please go through the inline replies.


5102.2: It seems that both positions can just be int. Another option is to
have two methods. Would it be clearer?

    InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
remoteLogSegmentMetadata,  int startPosition) throwsRemoteStorageException;

    InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
remoteLogSegmentMetadata, int startPosition, int endPosition) throws
RemoteStorageException;

That makes sense to me, updated the KIP.

6003: Could you also update the javadoc for the return value?

Updated.

6020: local.log.retention.bytes: Should it default to log.retention.bytes
to be consistent with local.log.retention.ms?

Yes, it can be defaulted to log.retention.bytes.

6021: Could you define TopicIdPartition?

Added TopicIdPartition in the KIP.

6022: For all public facing classes, could you specify the package name?

Updated.


Thanks,
Satish.

On Tue, Dec 8, 2020 at 12:59 AM Jun Rao <ju...@confluent.io> wrote:
>
> Hi, Satish,
>
> Thanks for the reply. A few more comments below.
>
> 5102.2: It seems that both positions can just be int. Another option is to
> have two methods. Would it be clearer?
>
>     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> remoteLogSegmentMetadata,
>                                     int startPosition) throws
> RemoteStorageException;
>
>     InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
> remoteLogSegmentMetadata,
>                                     int startPosition, int endPosition)
> throws RemoteStorageException;
>
> 6003: Could you also update the javadoc for the return value?
>
> 6010: What kind of tiering throughput have you seen with 5 threads?
>
> 6020: local.log.retention.bytes: Should it default to log.retention.bytes
> to be consistent with local.log.retention.ms?
>
> 6021: Could you define TopicIdPartition?
>
> 6022: For all public facing classes, could you specify the package name?
>
> It seems that you already added the topicId support. Two other remaining
> items are (a) the format of local tier metadata storage and (b) upgrade.
>
> Jun
>
> On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi Jun,
> > Thanks for your comments. Please find the inline replies below.
> >
> > >605.2 It's rare for the follower to need the remote data. So, the current
> > approach is fine too. Could you document the process of rebuilding the
> > producer state since we can't simply trim the producerState to an offset in
> > the middle of a segment.
> >
> > Will clarify in the KIP.
> >
> > >5102.2 Would it be clearer to make startPosiont long and endPosition of
> > Optional<Long>?
> >
> > We will have arg checks with respective validation. It is not a good
> > practice to have arguments with optional as mentioned here.
> > https://rules.sonarsource.com/java/RSPEC-3553
> >
> >
> > >5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > ByteBuffer.
> >
> > Updated.
> >
> > >5102.7 Could you define all public methods for LogSegmentData?
> >
> > Updated.
> >
> > >5103.5 Could you change the reference to rlm_process_interval_ms and
> > rlm_retry_interval_ms to the new config names? Also, the retry interval
> > config seems still missing. It would be useful to support exponential
> > backoff with the retry interval config.
> >
> > Good point. We wanted the retry with truncated exponential backoff,
> > updated the KIP.
> >
> > >5111. "RLM follower fetches the earliest offset for the earliest leader
> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
> > leaderEpoch) and updates that as the log start offset." This text is still
> > there. Also, could we remove earliestLogOffset() from RLMM?
> >
> > Updated.
> >
> > >5115. There are still references to "remote log cleaners".
> >
> > Updated.
> >
> > >6000. Since we are returning new error codes, we need to bump up the
> > protocol version for Fetch request. Also, it will be useful to document all
> > new error codes and whether they are retriable or not.
> >
> > Sure, we will add that in the KIP.
> >
> > >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently, leaderEpoch
> > is int32 instead of long.
> >
> > Updated.
> >
> > >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > RemoteLogSegmentMetadata.state()?
> >
> > No, it is fixed.
> >
> > >6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
> > topicPartition, long offset, int epochForOffset): Should this return
> > Optional<RemoteLogSegmentMetadata>?
> >
> > That makes sense, updated.
> >
> > >6005. RemoteLogState: It seems it's better to split it between
> > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the states
> > are never shared between the two use cases.
> >
> > Agree with that, updated.
> >
> > >6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However, is it
> > ture that other than the metadata topic, RLMM just needs to know whether
> > there is a replica assigned to this broker and doesn't need to know whether
> > the replica is the leader or the follower?
> >
> > That may be true. If the implementation does not need that, it can
> > ignore the information in the callback.
> >
> > >6007: "Handle expired remote segments (leader and follower)": Why is this
> > needed in both the leader and the follower?
> >
> > Updated.
> >
> > >6008.       "name": "SegmentSizeInBytes",
> >                 "type": "int64",
> > The segment size can just be int32.
> >
> > Updated.
> >
> > >6009. For the record format in the log, it seems that we need to add
> > record
> > type and record version before the serialized bytes. We can follow the
> > convention used in
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> >
> > Yes, KIP already mentions that these are serialized before the payload
> > as below. We will mention explicitly that these two are written before
> > the data is written.
> >
> > RLMM instance on broker publishes the message to the topic with key as
> > null and value with the below format.
> >
> > type      : unsigned var int, represents the value type. This value is
> > 'apikey' as mentioned in the schema.
> > version : unsigned var int, the 'version' number of the type as
> > mentioned in the schema.
> > data      : record payload in kafka protocol message format.
> >
> >
> > >6010. remote.log.manager.thread.pool.size: The default value is 10. This
> > might be too high when enabling the tiered feature for the first time.
> > Since there are lots of segments that need to be tiered initially, a large
> > number of threads could overwhelm the broker.
> >
> > Is the default value 5 reasonable?
> >
> > 6011. "The number of milli seconds to keep the local log segment before it
> > gets deleted. If not set, the value in `log.retention.minutes` is used. If
> > set to -1, no time limit is applied." We should use log.retention.ms
> > instead of log.retention.minutes.
> > Nice typo catch. Updated the KIP.
> >
> > Thanks,
> > Satish.
> >
> > On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Hi, Satish,
> > >
> > > Thanks for the updated KIP. A few more comments below.
> > >
> > > 605.2 It's rare for the follower to need the remote data. So, the current
> > > approach is fine too. Could you document the process of rebuilding the
> > > producer state since we can't simply trim the producerState to an offset
> > in
> > > the middle of a segment.
> > >
> > > 5102.2 Would it be clearer to make startPosiont long and endPosition of
> > > Optional<Long>?
> > >
> > > 5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > > ByteBuffer.
> > >
> > > 5102.7 Could you define all public methods for LogSegmentData?
> > >
> > > 5103.5 Could you change the reference to rlm_process_interval_ms and
> > > rlm_retry_interval_ms to the new config names? Also, the retry interval
> > > config seems still missing. It would be useful to support exponential
> > > backoff with the retry interval config.
> > >
> > > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> > int
> > > leaderEpoch) and updates that as the log start offset." This text is
> > still
> > > there. Also, could we remove earliestLogOffset() from RLMM?
> > >
> > > 5115. There are still references to "remote log cleaners".
> > >
> > > 6000. Since we are returning new error codes, we need to bump up the
> > > protocol version for Fetch request. Also, it will be useful to document
> > all
> > > new error codes and whether they are retriable or not.
> > >
> > > 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> > leaderEpoch
> > > is int32 instead of long.
> > >
> > > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > > RemoteLogSegmentMetadata.state()?
> > >
> > > 6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
> > > topicPartition, long offset, int epochForOffset): Should this return
> > > Optional<RemoteLogSegmentMetadata>?
> > >
> > > 6004. DeletePartitionUpdate.epoch(): It would be useful to pick a more
> > > indicative name so that people understand what epoch this is.
> > >
> > > 6005. RemoteLogState: It seems it's better to split it between
> > > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the states
> > > are never shared between the two use cases.
> > >
> > > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However, is it
> > > ture that other than the metadata topic, RLMM just needs to know whether
> > > there is a replica assigned to this broker and doesn't need to know
> > whether
> > > the replica is the leader or the follower?
> > >
> > > 6007: "Handle expired remote segments (leader and follower)": Why is this
> > > needed in both the leader and the follower?
> > >
> > > 6008.       "name": "SegmentSizeInBytes",
> > >                 "type": "int64",
> > > The segment size can just be int32.
> > >
> > > 6009. For the record format in the log, it seems that we need to add
> > record
> > > type and record version before the serialized bytes. We can follow the
> > > convention used in
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > > .
> > >
> > > 6010. remote.log.manager.thread.pool.size: The default value is 10. This
> > > might be too high when enabling the tiered feature for the first time.
> > > Since there are lots of segments that need to be tiered initially, a
> > large
> > > number of threads could overwhelm the broker.
> > >
> > > 6011. "The number of milli seconds to keep the local log segment before
> > it
> > > gets deleted. If not set, the value in `log.retention.minutes` is used.
> > If
> > > set to -1, no time limit is applied." We should use log.retention.ms
> > > instead of log.retention.minutes.
> > >
> > > Jun
> > >
> > > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <sa...@gmail.com>
> > > wrote:
> > >
> > > > Hi,
> > > > We updated the KIP with the points mentioned in the earlier mail
> > > > except for KIP-516 related changes. You can go through them and let us
> > > > know if you have any comments. We will update the KIP with the
> > > > remaining todo items and KIP-516 related changes by end of this
> > > > week(5th Dec).
> > > >
> > > > Thanks,
> > > > Satish.
> > > >
> > > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> > satish.duggana@gmail.com>
> > > > wrote:
> > > > >
> > > > > Hi Jun,
> > > > > Thanks for your comments. Please find the inline replies below.
> > > > >
> > > > > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > > > > sequence received from remote storage to [LSO, ELO]." I mentioned an
> > > > issue
> > > > > earlier. Suppose the leader's local start offset is 100. The follower
> > > > finds
> > > > > a remote segment covering offset range [80, 120). The producerState
> > with
> > > > > this remote segment is up to offset 120. To trim the producerState to
> > > > > offset 100 requires more work since one needs to download the
> > previous
> > > > > producerState up to offset 80 and then replay the messages from 80 to
> > > > 100.
> > > > > It seems that it's simpler in this case for the follower just to
> > take the
> > > > > remote segment as it is and start fetching from offset 120.
> > > > >
> > > > > We chose that approach to avoid any edge cases here. It may be
> > > > > possible that the remote log segment that is received may not have
> > the
> > > > > same leader epoch sequence from 100-120 as it contains on the
> > > > > leader(this can happen due to unclean leader). It is safe to start
> > > > > from what the leader returns here.Another way is to find the remote
> > > > > log segment
> > > > >
> > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > RLMM.onPartitionLeadershipChanges() is only called on the replicas
> > for a
> > > > > partition, not on the replicas for the __remote_log_segment_metadata
> > > > > partition. It's not clear how the leader of
> > __remote_log_segment_metadata
> > > > > obtains the metadata for remote segments for deletion.
> > > > >
> > > > > RLMM will always receive the callback for the remote log metadata
> > > > > topic partitions hosted on the local broker and these will be
> > > > > subscribed. I will make this clear in the KIP.
> > > > >
> > > > > 5100. KIP-516 has been accepted and is being implemented now. Could
> > you
> > > > > update the KIP based on topicID?
> > > > >
> > > > > We mentioned KIP-516 and how it helps. We will update this KIP with
> > > > > all the changes it brings with KIP-516.
> > > > >
> > > > > 5101. RLMM: It would be useful to clarify how the following two APIs
> > are
> > > > > used. According to the wiki, the former is used for topic deletion
> > and
> > > > the
> > > > > latter is used for retention. It seems that retention should use the
> > > > former
> > > > > since remote segments without a matching epoch in the leader
> > (potentially
> > > > > due to unclean leader election) also need to be garbage collected.
> > The
> > > > > latter seems to be used for the new leader to determine the last
> > tiered
> > > > > segment.
> > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > listRemoteLogSegments(TopicPartition
> > > > > topicPartition, long leaderEpoch);
> > > > >
> > > > > Right,.that is what we are currently doing. We will update the
> > > > > javadocs and wiki with that. Earlier, we did not want to remove the
> > > > > segments which are not matched with leader epochs from the ladder
> > > > > partition as they may be used later by a replica which can become a
> > > > > leader (unclean leader election) and refer those segments. But that
> > > > > may leak these segments in remote storage until the topic lifetime.
> > We
> > > > > decided to cleanup the segments with the oldest incase of size based
> > > > > retention also.
> > > > >
> > > > > 5102. RSM:
> > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > >
> > > > > It will be useful to have metadata for RSM to fetch log segment. It
> > > > > may create location/path using id with other metadata too.
> > > > >
> > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> > > > >
> > > > > Wanted to keep endPosition as optional to read till the end of the
> > > > > segment and avoid sentinels.
> > > > >
> > > > > 5102.3 Why only some of the methods have default implementation and
> > > > others
> > > > > Don't?
> > > > >
> > > > > Actually,  RSM will not have any default implementations. Those 3
> > > > > methods were made default earlier for tests etc. Updated the wiki.
> > > > >
> > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > and DeletePartitionUpdate?
> > > > >
> > > > > Sure, they will be added.
> > > > >
> > > > >
> > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file since
> > it
> > > > will
> > > > > be generated in memory.
> > > > >
> > > > > Right, this is in plan.
> > > > >
> > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > baseOffset
> > > > and
> > > > > startOffset. For example, deleteRecords() could move the startOffset
> > to
> > > > the
> > > > > middle of a segment. If we copy the full segment to remote storage,
> > the
> > > > > baseOffset and the startOffset will be different.
> > > > >
> > > > > Good point. startOffset is baseOffset by default, if not set
> > explicitly.
> > > > >
> > > > > 5102.7 Could we define all the public methods for
> > > > RemoteLogSegmentMetadata
> > > > > and LogSegmentData?
> > > > >
> > > > > Sure, updated the wiki.
> > > > >
> > > > > 5102.8 Could we document whether endOffset in
> > RemoteLogSegmentMetadata is
> > > > > inclusive/exclusive?
> > > > >
> > > > > It is inclusive, will update.
> > > > >
> > > > > 5103. configs:
> > > > > 5103.1 Could we define the default value of non-required configs
> > (e.g the
> > > > > size of new thread pools)?
> > > > >
> > > > > Sure, that makes sense.
> > > > >
> > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > > retention.ms,
> > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > >
> > > > > Right, we do not have  remote.log.retention as we discussed earlier.
> > > > > Thanks for catching the typo.
> > > > >
> > > > > 5103.3 remote.log.manager.thread.pool.size: The description says
> > "used in
> > > > > scheduling tasks to copy segments, fetch remote log indexes and
> > clean up
> > > > > remote log segments". However, there is a separate
> > > > > config remote.log.reader.threads for fetching remote data. It's
> > weird to
> > > > > fetch remote index and log in different thread pools since both are
> > used
> > > > > for serving fetch requests.
> > > > >
> > > > > Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
> > > > > activities. Fetch path always goes through remote.log.reader.threads.
> > > > >
> > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of
> > time
> > > > to
> > > > > back off when there is no work to do? If so, perhaps it can be
> > renamed as
> > > > > backoff.ms.
> > > > >
> > > > > This is the delay interval for each iteration. It may be renamed to
> > > > > remote.log.manager.task.delay.ms
> > > > >
> > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > configs? If
> > > > > so, they need to be listed in this section.
> > > > >
> > > > > remote.log.manager.task.interval.ms is the process internal, retry
> > > > > interval is missing in the configs, which will be updated in the KIP.
> > > > >
> > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> > files of
> > > > > remote log segments to avoid multiple index fetches from the remote
> > > > > storage." Is the RLM in memory or on disk? If on disk, where is it
> > > > stored?
> > > > > Do we need a configuration to bound the size?
> > > > >
> > > > > It is stored on disk. They are stored in a directory
> > > > > `remote-log-index-cache` under log dir. We plan to have a config for
> > > > > that instead of default. We will have a configuration for that.
> > > > >
> > > > > 5105. The KIP uses local-log-start-offset and Earliest Local Offset
> > in
> > > > > different places. It would be useful to standardize the terminology.
> > > > >
> > > > > Sure.
> > > > >
> > > > > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> > > > options
> > > > > without saying which option is chosen.
> > > > > We already mentioned in the KIP that we chose option-2.
> > > > >
> > > > > 5107. Follower to leader transition: It has step 2, but not step 1.
> > > > > Step-1 is there but it is not explicitly highlighted. It is previous
> > > > > table to step-2.
> > > > >
> > > > > 5108. If a consumer fetches from the remote data and the remote
> > storage
> > > > is
> > > > > not available, what error code is used in the fetch response?
> > > > >
> > > > > Good point. We have not yet defined the error for this case. We need
> > > > > to define an error message and send the same in fetch response.
> > > > >
> > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > > > > offset whose timestamp is >= to the given timestamp in the request.
> > That
> > > > > means it checks in remote log time indexes first, after which local
> > log
> > > > > time indexes are checked." Could you document which method in RLMM is
> > > > used
> > > > > for this?
> > > > >
> > > > > Okay.
> > > > >
> > > > > 5110. Stopreplica: "it sets all the remote log segment metadata of
> > that
> > > > > partition with a delete marker and publishes them to RLMM." This
> > seems
> > > > > outdated given the new topic deletion logic.
> > > > >
> > > > > Will update with KIP-516 related points.
> > > > >
> > > > > 5111. "RLM follower fetches the earliest offset for the earliest
> > leader
> > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > topicPartition,
> > > > int
> > > > > leaderEpoch) and updates that as the log start offset." Do we need
> > that
> > > > > since replication propagates logStartOffset already?
> > > > >
> > > > > Good point. Right, existing replication protocol takes care of
> > > > > updating the followers’s log start offset received from the leader.
> > > > >
> > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching from
> > remote
> > > > > storage?
> > > > >
> > > > > Remote reads may fail within the current default wait time, but
> > > > > subsequent fetches would be able to serve as that data is stored in
> > > > > the local cache. This cache is currently implemented in RSMs. But we
> > > > > plan to pull this into the remote log messaging layer in future.
> > > > >
> > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > reading
> > > > the
> > > > > messages again when a broker is restarted." Could you describe the
> > format
> > > > > and the location of the file? Also, could the same message be
> > processed
> > > > by
> > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > >
> > > > > Sure, we will update in the KIP.
> > > > >
> > > > > 5114. Message format
> > > > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord
> > with
> > > > > apiKey 0 and 1.
> > > > >
> > > > > Nice catch, that was a typo. Fixed in the wiki.
> > > > >
> > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> > > > endOffset
> > > > > is inclusive/exclusive?
> > > > > It is inclusive, will update.
> > > > >
> > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch
> > a
> > > > bit
> > > > > more? Is that the epoch of the leader when it copies the segment to
> > > > remote
> > > > > storage? Also, how will this field be used?
> > > > >
> > > > > Right, this is the leader epoch of the broker which copied this
> > > > > segment. This is helpful in reason about which broker copied the
> > > > > segment to remote storage.
> > > > >
> > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> > record in
> > > > > Kafka already has a timestamp field. Could we just use that?
> > > > >
> > > > > This is the  timestamp at which  the respective event occurred. Added
> > > > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > > > implementation. We thought about that but it looked cleaner to use at
> > > > > the message structure level instead of getting that from the consumer
> > > > > record and using that to build the respective event.
> > > > >
> > > > >
> > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > >
> > > > > Right, it looks like config allows only int value >= 14.
> > > > >
> > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the log
> > cleaner
> > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > RemotePartitionRemover.
> > > > >
> > > > > I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we
> > > > > have other manager classes like RLM, RLMM).
> > > > >
> > > > > 5116. "RLC receives the delete_partition_marked and processes it if
> > it is
> > > > > not yet processed earlier." How does it know whether
> > > > > delete_partition_marked has been processed earlier?
> > > > >
> > > > > This is to handle duplicate delete_partition_marked events. RLC
> > > > > internally maintains a state for the delete_partition events and if
> > it
> > > > > already has an existing event then it ignores if it is already being
> > > > > processed.
> > > > >
> > > > > 5117. Should we add a new MessageFormatter to read the tier metadata
> > > > topic?
> > > > >
> > > > > Right, this is in plan but did not mention it in the KIP. This will
> > be
> > > > > useful for debugging purposes too.
> > > > >
> > > > > 5118. "Maximum remote log reader thread pool task queue size. If the
> > task
> > > > > queue is full, broker will stop reading remote log segments." What
> > do we
> > > > > return to the fetch request in this case?
> > > > >
> > > > > We return an error response for that partition.
> > > > >
> > > > > 5119. It would be useful to list all things not supported in the
> > first
> > > > > version in a Future work or Limitations section. For example,
> > compacted
> > > > > topic, JBOD, changing remote.log.storage.enable from true to false,
> > etc.
> > > > >
> > > > > We already have a non-goals section which is filled with some of
> > these
> > > > > details. Do we need another limitations section?
> > > > >
> > > > > Thanks,
> > > > > Satish.
> > > > >
> > > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > Hi, Satish,
> > > > > >
> > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > >
> > > > > > 605.2 "Build the local leader epoch cache by cutting the leader
> > epoch
> > > > > > sequence received from remote storage to [LSO, ELO]." I mentioned
> > an
> > > > issue
> > > > > > earlier. Suppose the leader's local start offset is 100. The
> > follower
> > > > finds
> > > > > > a remote segment covering offset range [80, 120). The producerState
> > > > with
> > > > > > this remote segment is up to offset 120. To trim the producerState
> > to
> > > > > > offset 100 requires more work since one needs to download the
> > previous
> > > > > > producerState up to offset 80 and then replay the messages from 80
> > to
> > > > 100.
> > > > > > It seems that it's simpler in this case for the follower just to
> > take
> > > > the
> > > > > > remote segment as it is and start fetching from offset 120.
> > > > > >
> > > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > > RLMM.onPartitionLeadershipChanges() is only called on the replicas
> > for
> > > > a
> > > > > > partition, not on the replicas for the
> > __remote_log_segment_metadata
> > > > > > partition. It's not clear how the leader of
> > > > __remote_log_segment_metadata
> > > > > > obtains the metadata for remote segments for deletion.
> > > > > >
> > > > > > 5100. KIP-516 has been accepted and is being implemented now.
> > Could you
> > > > > > update the KIP based on topicID?
> > > > > >
> > > > > > 5101. RLMM: It would be useful to clarify how the following two
> > APIs
> > > > are
> > > > > > used. According to the wiki, the former is used for topic deletion
> > and
> > > > the
> > > > > > latter is used for retention. It seems that retention should use
> > the
> > > > former
> > > > > > since remote segments without a matching epoch in the leader
> > > > (potentially
> > > > > > due to unclean leader election) also need to be garbage collected.
> > The
> > > > > > latter seems to be used for the new leader to determine the last
> > tiered
> > > > > > segment.
> > > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > > >     Iterator<RemoteLogSegmentMetadata>
> > > > listRemoteLogSegments(TopicPartition
> > > > > > topicPartition, long leaderEpoch);
> > > > > >
> > > > > > 5102. RSM:
> > > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that they
> > can
> > > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of
> > Long?
> > > > > > 5102.3 Why only some of the methods have default implementation and
> > > > others
> > > > > > don't?
> > > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > > and DeletePartitionUpdate?
> > > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> > since it
> > > > will
> > > > > > be generated in memory.
> > > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > > baseOffset and
> > > > > > startOffset. For example, deleteRecords() could move the
> > startOffset
> > > > to the
> > > > > > middle of a segment. If we copy the full segment to remote
> > storage, the
> > > > > > baseOffset and the startOffset will be different.
> > > > > > 5102.7 Could we define all the public methods for
> > > > RemoteLogSegmentMetadata
> > > > > > and LogSegmentData?
> > > > > > 5102.8 Could we document whether endOffset in
> > RemoteLogSegmentMetadata
> > > > is
> > > > > > inclusive/exclusive?
> > > > > >
> > > > > > 5103. configs:
> > > > > > 5103.1 Could we define the default value of non-required configs
> > (e.g
> > > > the
> > > > > > size of new thread pools)?
> > > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > > retention.ms,
> > > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > > > 5103.3 remote.log.manager.thread.pool.size: The description says
> > "used
> > > > in
> > > > > > scheduling tasks to copy segments, fetch remote log indexes and
> > clean
> > > > up
> > > > > > remote log segments". However, there is a separate
> > > > > > config remote.log.reader.threads for fetching remote data. It's
> > weird
> > > > to
> > > > > > fetch remote index and log in different thread pools since both are
> > > > used
> > > > > > for serving fetch requests.
> > > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of
> > > > time to
> > > > > > back off when there is no work to do? If so, perhaps it can be
> > renamed
> > > > as
> > > > > > backoff.ms.
> > > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> > configs?
> > > > If
> > > > > > so, they need to be listed in this section.
> > > > > >
> > > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> > files
> > > > of
> > > > > > remote log segments to avoid multiple index fetches from the remote
> > > > > > storage." Is the RLM in memory or on disk? If on disk, where is it
> > > > stored?
> > > > > > Do we need a configuration to bound the size?
> > > > > >
> > > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> > Offset in
> > > > > > different places. It would be useful to standardize the
> > terminology.
> > > > > >
> > > > > > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> > > > options
> > > > > > without saying which option is chosen.
> > > > > >
> > > > > > 5107. Follower to leader transition: It has step 2, but not step 1.
> > > > > >
> > > > > > 5108. If a consumer fetches from the remote data and the remote
> > > > storage is
> > > > > > not available, what error code is used in the fetch response?
> > > > > >
> > > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> > message
> > > > > > offset whose timestamp is >= to the given timestamp in the request.
> > > > That
> > > > > > means it checks in remote log time indexes first, after which
> > local log
> > > > > > time indexes are checked." Could you document which method in RLMM
> > is
> > > > used
> > > > > > for this?
> > > > > >
> > > > > > 5110. Stopreplica: "it sets all the remote log segment metadata of
> > that
> > > > > > partition with a delete marker and publishes them to RLMM." This
> > seems
> > > > > > outdated given the new topic deletion logic.
> > > > > >
> > > > > > 5111. "RLM follower fetches the earliest offset for the earliest
> > leader
> > > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> > topicPartition,
> > > > int
> > > > > > leaderEpoch) and updates that as the log start offset." Do we need
> > that
> > > > > > since replication propagates logStartOffset already?
> > > > > >
> > > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching from
> > remote
> > > > > > storage?
> > > > > >
> > > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > > > reading the
> > > > > > messages again when a broker is restarted." Could you describe the
> > > > format
> > > > > > and the location of the file? Also, could the same message be
> > > > processed by
> > > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > > >
> > > > > > 5114. Message format
> > > > > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord
> > with
> > > > > > apiKey 0 and 1.
> > > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> > > > endOffset
> > > > > > is inclusive/exclusive?
> > > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> > LeaderEpoch a
> > > > bit
> > > > > > more? Is that the epoch of the leader when it copies the segment to
> > > > remote
> > > > > > storage? Also, how will this field be used?
> > > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> > record
> > > > in
> > > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > > >
> > > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the log
> > > > cleaner
> > > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > > RemotePartitionRemover.
> > > > > >
> > > > > > 5116. "RLC receives the delete_partition_marked and processes it
> > if it
> > > > is
> > > > > > not yet processed earlier." How does it know whether
> > > > > > delete_partition_marked has been processed earlier?
> > > > > >
> > > > > > 5117. Should we add a new MessageFormatter to read the tier
> > metadata
> > > > topic?
> > > > > >
> > > > > > 5118. "Maximum remote log reader thread pool task queue size. If
> > the
> > > > task
> > > > > > queue is full, broker will stop reading remote log segments." What
> > do
> > > > we
> > > > > > return to the fetch request in this case?
> > > > > >
> > > > > > 5119. It would be useful to list all things not supported in the
> > first
> > > > > > version in a Future work or Limitations section. For example,
> > compacted
> > > > > > topic, JBOD, changing remote.log.storage.enable from true to false,
> > > > etc.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > > kprakasam@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.duggana@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.duggana@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 <
> > > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > > (
> > > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > > >>>>>>>
> > > > > > > >
> > > >
> > > >
> >

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

Posted by Jun Rao <ju...@confluent.io>.
Hi, Satish,

Thanks for the reply. A few more comments below.

5102.2: It seems that both positions can just be int. Another option is to
have two methods. Would it be clearer?

    InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
remoteLogSegmentMetadata,
                                    int startPosition) throws
RemoteStorageException;

    InputStream fetchLogSegmentData(RemoteLogSegmentMetadata
remoteLogSegmentMetadata,
                                    int startPosition, int endPosition)
throws RemoteStorageException;

6003: Could you also update the javadoc for the return value?

6010: What kind of tiering throughput have you seen with 5 threads?

6020: local.log.retention.bytes: Should it default to log.retention.bytes
to be consistent with local.log.retention.ms?

6021: Could you define TopicIdPartition?

6022: For all public facing classes, could you specify the package name?

It seems that you already added the topicId support. Two other remaining
items are (a) the format of local tier metadata storage and (b) upgrade.

Jun

On Mon, Dec 7, 2020 at 8:56 AM Satish Duggana <sa...@gmail.com>
wrote:

> Hi Jun,
> Thanks for your comments. Please find the inline replies below.
>
> >605.2 It's rare for the follower to need the remote data. So, the current
> approach is fine too. Could you document the process of rebuilding the
> producer state since we can't simply trim the producerState to an offset in
> the middle of a segment.
>
> Will clarify in the KIP.
>
> >5102.2 Would it be clearer to make startPosiont long and endPosition of
> Optional<Long>?
>
> We will have arg checks with respective validation. It is not a good
> practice to have arguments with optional as mentioned here.
> https://rules.sonarsource.com/java/RSPEC-3553
>
>
> >5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> ByteBuffer.
>
> Updated.
>
> >5102.7 Could you define all public methods for LogSegmentData?
>
> Updated.
>
> >5103.5 Could you change the reference to rlm_process_interval_ms and
> rlm_retry_interval_ms to the new config names? Also, the retry interval
> config seems still missing. It would be useful to support exponential
> backoff with the retry interval config.
>
> Good point. We wanted the retry with truncated exponential backoff,
> updated the KIP.
>
> >5111. "RLM follower fetches the earliest offset for the earliest leader
> epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
> leaderEpoch) and updates that as the log start offset." This text is still
> there. Also, could we remove earliestLogOffset() from RLMM?
>
> Updated.
>
> >5115. There are still references to "remote log cleaners".
>
> Updated.
>
> >6000. Since we are returning new error codes, we need to bump up the
> protocol version for Fetch request. Also, it will be useful to document all
> new error codes and whether they are retriable or not.
>
> Sure, we will add that in the KIP.
>
> >6001. public Map<Long, Long> segmentLeaderEpochs(): Currently, leaderEpoch
> is int32 instead of long.
>
> Updated.
>
> >6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> RemoteLogSegmentMetadata.state()?
>
> No, it is fixed.
>
> >6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
> topicPartition, long offset, int epochForOffset): Should this return
> Optional<RemoteLogSegmentMetadata>?
>
> That makes sense, updated.
>
> >6005. RemoteLogState: It seems it's better to split it between
> DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the states
> are never shared between the two use cases.
>
> Agree with that, updated.
>
> >6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However, is it
> ture that other than the metadata topic, RLMM just needs to know whether
> there is a replica assigned to this broker and doesn't need to know whether
> the replica is the leader or the follower?
>
> That may be true. If the implementation does not need that, it can
> ignore the information in the callback.
>
> >6007: "Handle expired remote segments (leader and follower)": Why is this
> needed in both the leader and the follower?
>
> Updated.
>
> >6008.       "name": "SegmentSizeInBytes",
>                 "type": "int64",
> The segment size can just be int32.
>
> Updated.
>
> >6009. For the record format in the log, it seems that we need to add
> record
> type and record version before the serialized bytes. We can follow the
> convention used in
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
>
> Yes, KIP already mentions that these are serialized before the payload
> as below. We will mention explicitly that these two are written before
> the data is written.
>
> RLMM instance on broker publishes the message to the topic with key as
> null and value with the below format.
>
> type      : unsigned var int, represents the value type. This value is
> 'apikey' as mentioned in the schema.
> version : unsigned var int, the 'version' number of the type as
> mentioned in the schema.
> data      : record payload in kafka protocol message format.
>
>
> >6010. remote.log.manager.thread.pool.size: The default value is 10. This
> might be too high when enabling the tiered feature for the first time.
> Since there are lots of segments that need to be tiered initially, a large
> number of threads could overwhelm the broker.
>
> Is the default value 5 reasonable?
>
> 6011. "The number of milli seconds to keep the local log segment before it
> gets deleted. If not set, the value in `log.retention.minutes` is used. If
> set to -1, no time limit is applied." We should use log.retention.ms
> instead of log.retention.minutes.
> Nice typo catch. Updated the KIP.
>
> Thanks,
> Satish.
>
> On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
> >
> > Hi, Satish,
> >
> > Thanks for the updated KIP. A few more comments below.
> >
> > 605.2 It's rare for the follower to need the remote data. So, the current
> > approach is fine too. Could you document the process of rebuilding the
> > producer state since we can't simply trim the producerState to an offset
> in
> > the middle of a segment.
> >
> > 5102.2 Would it be clearer to make startPosiont long and endPosition of
> > Optional<Long>?
> >
> > 5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> > ByteBuffer.
> >
> > 5102.7 Could you define all public methods for LogSegmentData?
> >
> > 5103.5 Could you change the reference to rlm_process_interval_ms and
> > rlm_retry_interval_ms to the new config names? Also, the retry interval
> > config seems still missing. It would be useful to support exponential
> > backoff with the retry interval config.
> >
> > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> int
> > leaderEpoch) and updates that as the log start offset." This text is
> still
> > there. Also, could we remove earliestLogOffset() from RLMM?
> >
> > 5115. There are still references to "remote log cleaners".
> >
> > 6000. Since we are returning new error codes, we need to bump up the
> > protocol version for Fetch request. Also, it will be useful to document
> all
> > new error codes and whether they are retriable or not.
> >
> > 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently,
> leaderEpoch
> > is int32 instead of long.
> >
> > 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> > RemoteLogSegmentMetadata.state()?
> >
> > 6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
> > topicPartition, long offset, int epochForOffset): Should this return
> > Optional<RemoteLogSegmentMetadata>?
> >
> > 6004. DeletePartitionUpdate.epoch(): It would be useful to pick a more
> > indicative name so that people understand what epoch this is.
> >
> > 6005. RemoteLogState: It seems it's better to split it between
> > DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the states
> > are never shared between the two use cases.
> >
> > 6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However, is it
> > ture that other than the metadata topic, RLMM just needs to know whether
> > there is a replica assigned to this broker and doesn't need to know
> whether
> > the replica is the leader or the follower?
> >
> > 6007: "Handle expired remote segments (leader and follower)": Why is this
> > needed in both the leader and the follower?
> >
> > 6008.       "name": "SegmentSizeInBytes",
> >                 "type": "int64",
> > The segment size can just be int32.
> >
> > 6009. For the record format in the log, it seems that we need to add
> record
> > type and record version before the serialized bytes. We can follow the
> > convention used in
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> > .
> >
> > 6010. remote.log.manager.thread.pool.size: The default value is 10. This
> > might be too high when enabling the tiered feature for the first time.
> > Since there are lots of segments that need to be tiered initially, a
> large
> > number of threads could overwhelm the broker.
> >
> > 6011. "The number of milli seconds to keep the local log segment before
> it
> > gets deleted. If not set, the value in `log.retention.minutes` is used.
> If
> > set to -1, no time limit is applied." We should use log.retention.ms
> > instead of log.retention.minutes.
> >
> > Jun
> >
> > On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <sa...@gmail.com>
> > wrote:
> >
> > > Hi,
> > > We updated the KIP with the points mentioned in the earlier mail
> > > except for KIP-516 related changes. You can go through them and let us
> > > know if you have any comments. We will update the KIP with the
> > > remaining todo items and KIP-516 related changes by end of this
> > > week(5th Dec).
> > >
> > > Thanks,
> > > Satish.
> > >
> > > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <
> satish.duggana@gmail.com>
> > > wrote:
> > > >
> > > > Hi Jun,
> > > > Thanks for your comments. Please find the inline replies below.
> > > >
> > > > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > > > sequence received from remote storage to [LSO, ELO]." I mentioned an
> > > issue
> > > > earlier. Suppose the leader's local start offset is 100. The follower
> > > finds
> > > > a remote segment covering offset range [80, 120). The producerState
> with
> > > > this remote segment is up to offset 120. To trim the producerState to
> > > > offset 100 requires more work since one needs to download the
> previous
> > > > producerState up to offset 80 and then replay the messages from 80 to
> > > 100.
> > > > It seems that it's simpler in this case for the follower just to
> take the
> > > > remote segment as it is and start fetching from offset 120.
> > > >
> > > > We chose that approach to avoid any edge cases here. It may be
> > > > possible that the remote log segment that is received may not have
> the
> > > > same leader epoch sequence from 100-120 as it contains on the
> > > > leader(this can happen due to unclean leader). It is safe to start
> > > > from what the leader returns here.Another way is to find the remote
> > > > log segment
> > > >
> > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > RLMM.onPartitionLeadershipChanges() is only called on the replicas
> for a
> > > > partition, not on the replicas for the __remote_log_segment_metadata
> > > > partition. It's not clear how the leader of
> __remote_log_segment_metadata
> > > > obtains the metadata for remote segments for deletion.
> > > >
> > > > RLMM will always receive the callback for the remote log metadata
> > > > topic partitions hosted on the local broker and these will be
> > > > subscribed. I will make this clear in the KIP.
> > > >
> > > > 5100. KIP-516 has been accepted and is being implemented now. Could
> you
> > > > update the KIP based on topicID?
> > > >
> > > > We mentioned KIP-516 and how it helps. We will update this KIP with
> > > > all the changes it brings with KIP-516.
> > > >
> > > > 5101. RLMM: It would be useful to clarify how the following two APIs
> are
> > > > used. According to the wiki, the former is used for topic deletion
> and
> > > the
> > > > latter is used for retention. It seems that retention should use the
> > > former
> > > > since remote segments without a matching epoch in the leader
> (potentially
> > > > due to unclean leader election) also need to be garbage collected.
> The
> > > > latter seems to be used for the new leader to determine the last
> tiered
> > > > segment.
> > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > >     Iterator<RemoteLogSegmentMetadata>
> > > listRemoteLogSegments(TopicPartition
> > > > topicPartition, long leaderEpoch);
> > > >
> > > > Right,.that is what we are currently doing. We will update the
> > > > javadocs and wiki with that. Earlier, we did not want to remove the
> > > > segments which are not matched with leader epochs from the ladder
> > > > partition as they may be used later by a replica which can become a
> > > > leader (unclean leader election) and refer those segments. But that
> > > > may leak these segments in remote storage until the topic lifetime.
> We
> > > > decided to cleanup the segments with the oldest incase of size based
> > > > retention also.
> > > >
> > > > 5102. RSM:
> > > > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > >
> > > > It will be useful to have metadata for RSM to fetch log segment. It
> > > > may create location/path using id with other metadata too.
> > > >
> > > > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> > > >
> > > > Wanted to keep endPosition as optional to read till the end of the
> > > > segment and avoid sentinels.
> > > >
> > > > 5102.3 Why only some of the methods have default implementation and
> > > others
> > > > Don't?
> > > >
> > > > Actually,  RSM will not have any default implementations. Those 3
> > > > methods were made default earlier for tests etc. Updated the wiki.
> > > >
> > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > and DeletePartitionUpdate?
> > > >
> > > > Sure, they will be added.
> > > >
> > > >
> > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > in leaderEpochIndex as a ByteBuffer or byte array than a file since
> it
> > > will
> > > > be generated in memory.
> > > >
> > > > Right, this is in plan.
> > > >
> > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> baseOffset
> > > and
> > > > startOffset. For example, deleteRecords() could move the startOffset
> to
> > > the
> > > > middle of a segment. If we copy the full segment to remote storage,
> the
> > > > baseOffset and the startOffset will be different.
> > > >
> > > > Good point. startOffset is baseOffset by default, if not set
> explicitly.
> > > >
> > > > 5102.7 Could we define all the public methods for
> > > RemoteLogSegmentMetadata
> > > > and LogSegmentData?
> > > >
> > > > Sure, updated the wiki.
> > > >
> > > > 5102.8 Could we document whether endOffset in
> RemoteLogSegmentMetadata is
> > > > inclusive/exclusive?
> > > >
> > > > It is inclusive, will update.
> > > >
> > > > 5103. configs:
> > > > 5103.1 Could we define the default value of non-required configs
> (e.g the
> > > > size of new thread pools)?
> > > >
> > > > Sure, that makes sense.
> > > >
> > > > 5103.2 It seems that local.log.retention.ms should default to
> > > retention.ms,
> > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > that local.log.retention.bytes should default to segment.bytes.
> > > >
> > > > Right, we do not have  remote.log.retention as we discussed earlier.
> > > > Thanks for catching the typo.
> > > >
> > > > 5103.3 remote.log.manager.thread.pool.size: The description says
> "used in
> > > > scheduling tasks to copy segments, fetch remote log indexes and
> clean up
> > > > remote log segments". However, there is a separate
> > > > config remote.log.reader.threads for fetching remote data. It's
> weird to
> > > > fetch remote index and log in different thread pools since both are
> used
> > > > for serving fetch requests.
> > > >
> > > > Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
> > > > activities. Fetch path always goes through remote.log.reader.threads.
> > > >
> > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of
> time
> > > to
> > > > back off when there is no work to do? If so, perhaps it can be
> renamed as
> > > > backoff.ms.
> > > >
> > > > This is the delay interval for each iteration. It may be renamed to
> > > > remote.log.manager.task.delay.ms
> > > >
> > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> configs? If
> > > > so, they need to be listed in this section.
> > > >
> > > > remote.log.manager.task.interval.ms is the process internal, retry
> > > > interval is missing in the configs, which will be updated in the KIP.
> > > >
> > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> files of
> > > > remote log segments to avoid multiple index fetches from the remote
> > > > storage." Is the RLM in memory or on disk? If on disk, where is it
> > > stored?
> > > > Do we need a configuration to bound the size?
> > > >
> > > > It is stored on disk. They are stored in a directory
> > > > `remote-log-index-cache` under log dir. We plan to have a config for
> > > > that instead of default. We will have a configuration for that.
> > > >
> > > > 5105. The KIP uses local-log-start-offset and Earliest Local Offset
> in
> > > > different places. It would be useful to standardize the terminology.
> > > >
> > > > Sure.
> > > >
> > > > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> > > options
> > > > without saying which option is chosen.
> > > > We already mentioned in the KIP that we chose option-2.
> > > >
> > > > 5107. Follower to leader transition: It has step 2, but not step 1.
> > > > Step-1 is there but it is not explicitly highlighted. It is previous
> > > > table to step-2.
> > > >
> > > > 5108. If a consumer fetches from the remote data and the remote
> storage
> > > is
> > > > not available, what error code is used in the fetch response?
> > > >
> > > > Good point. We have not yet defined the error for this case. We need
> > > > to define an error message and send the same in fetch response.
> > > >
> > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > > > offset whose timestamp is >= to the given timestamp in the request.
> That
> > > > means it checks in remote log time indexes first, after which local
> log
> > > > time indexes are checked." Could you document which method in RLMM is
> > > used
> > > > for this?
> > > >
> > > > Okay.
> > > >
> > > > 5110. Stopreplica: "it sets all the remote log segment metadata of
> that
> > > > partition with a delete marker and publishes them to RLMM." This
> seems
> > > > outdated given the new topic deletion logic.
> > > >
> > > > Will update with KIP-516 related points.
> > > >
> > > > 5111. "RLM follower fetches the earliest offset for the earliest
> leader
> > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> topicPartition,
> > > int
> > > > leaderEpoch) and updates that as the log start offset." Do we need
> that
> > > > since replication propagates logStartOffset already?
> > > >
> > > > Good point. Right, existing replication protocol takes care of
> > > > updating the followers’s log start offset received from the leader.
> > > >
> > > > 5112. Is the default maxWaitMs of 500ms enough for fetching from
> remote
> > > > storage?
> > > >
> > > > Remote reads may fail within the current default wait time, but
> > > > subsequent fetches would be able to serve as that data is stored in
> > > > the local cache. This cache is currently implemented in RSMs. But we
> > > > plan to pull this into the remote log messaging layer in future.
> > > >
> > > > 5113. "Committed offsets can be stored in a local file to avoid
> reading
> > > the
> > > > messages again when a broker is restarted." Could you describe the
> format
> > > > and the location of the file? Also, could the same message be
> processed
> > > by
> > > > RLMM again after broker restart? If so, how do we handle that?
> > > >
> > > > Sure, we will update in the KIP.
> > > >
> > > > 5114. Message format
> > > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord
> with
> > > > apiKey 0 and 1.
> > > >
> > > > Nice catch, that was a typo. Fixed in the wiki.
> > > >
> > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> > > endOffset
> > > > is inclusive/exclusive?
> > > > It is inclusive, will update.
> > > >
> > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch
> a
> > > bit
> > > > more? Is that the epoch of the leader when it copies the segment to
> > > remote
> > > > storage? Also, how will this field be used?
> > > >
> > > > Right, this is the leader epoch of the broker which copied this
> > > > segment. This is helpful in reason about which broker copied the
> > > > segment to remote storage.
> > > >
> > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> record in
> > > > Kafka already has a timestamp field. Could we just use that?
> > > >
> > > > This is the  timestamp at which  the respective event occurred. Added
> > > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > > implementation. We thought about that but it looked cleaner to use at
> > > > the message structure level instead of getting that from the consumer
> > > > record and using that to build the respective event.
> > > >
> > > >
> > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > >
> > > > Right, it looks like config allows only int value >= 14.
> > > >
> > > > 5115. RemoteLogCleaner(RLC): This could be confused with the log
> cleaner
> > > > for compaction. Perhaps it can be renamed to sth like
> > > > RemotePartitionRemover.
> > > >
> > > > I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we
> > > > have other manager classes like RLM, RLMM).
> > > >
> > > > 5116. "RLC receives the delete_partition_marked and processes it if
> it is
> > > > not yet processed earlier." How does it know whether
> > > > delete_partition_marked has been processed earlier?
> > > >
> > > > This is to handle duplicate delete_partition_marked events. RLC
> > > > internally maintains a state for the delete_partition events and if
> it
> > > > already has an existing event then it ignores if it is already being
> > > > processed.
> > > >
> > > > 5117. Should we add a new MessageFormatter to read the tier metadata
> > > topic?
> > > >
> > > > Right, this is in plan but did not mention it in the KIP. This will
> be
> > > > useful for debugging purposes too.
> > > >
> > > > 5118. "Maximum remote log reader thread pool task queue size. If the
> task
> > > > queue is full, broker will stop reading remote log segments." What
> do we
> > > > return to the fetch request in this case?
> > > >
> > > > We return an error response for that partition.
> > > >
> > > > 5119. It would be useful to list all things not supported in the
> first
> > > > version in a Future work or Limitations section. For example,
> compacted
> > > > topic, JBOD, changing remote.log.storage.enable from true to false,
> etc.
> > > >
> > > > We already have a non-goals section which is filled with some of
> these
> > > > details. Do we need another limitations section?
> > > >
> > > > Thanks,
> > > > Satish.
> > > >
> > > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the updated KIP. A few more comments below.
> > > > >
> > > > > 605.2 "Build the local leader epoch cache by cutting the leader
> epoch
> > > > > sequence received from remote storage to [LSO, ELO]." I mentioned
> an
> > > issue
> > > > > earlier. Suppose the leader's local start offset is 100. The
> follower
> > > finds
> > > > > a remote segment covering offset range [80, 120). The producerState
> > > with
> > > > > this remote segment is up to offset 120. To trim the producerState
> to
> > > > > offset 100 requires more work since one needs to download the
> previous
> > > > > producerState up to offset 80 and then replay the messages from 80
> to
> > > 100.
> > > > > It seems that it's simpler in this case for the follower just to
> take
> > > the
> > > > > remote segment as it is and start fetching from offset 120.
> > > > >
> > > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > > RLMM.onPartitionLeadershipChanges() is only called on the replicas
> for
> > > a
> > > > > partition, not on the replicas for the
> __remote_log_segment_metadata
> > > > > partition. It's not clear how the leader of
> > > __remote_log_segment_metadata
> > > > > obtains the metadata for remote segments for deletion.
> > > > >
> > > > > 5100. KIP-516 has been accepted and is being implemented now.
> Could you
> > > > > update the KIP based on topicID?
> > > > >
> > > > > 5101. RLMM: It would be useful to clarify how the following two
> APIs
> > > are
> > > > > used. According to the wiki, the former is used for topic deletion
> and
> > > the
> > > > > latter is used for retention. It seems that retention should use
> the
> > > former
> > > > > since remote segments without a matching epoch in the leader
> > > (potentially
> > > > > due to unclean leader election) also need to be garbage collected.
> The
> > > > > latter seems to be used for the new leader to determine the last
> tiered
> > > > > segment.
> > > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > > >     Iterator<RemoteLogSegmentMetadata>
> > > listRemoteLogSegments(TopicPartition
> > > > > topicPartition, long leaderEpoch);
> > > > >
> > > > > 5102. RSM:
> > > > > 5102.1 For methods like fetchLogSegmentData(), it seems that they
> can
> > > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > > 5102.2 In fetchLogSegmentData(), should we use long instead of
> Long?
> > > > > 5102.3 Why only some of the methods have default implementation and
> > > others
> > > > > don't?
> > > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > > and DeletePartitionUpdate?
> > > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > > in leaderEpochIndex as a ByteBuffer or byte array than a file
> since it
> > > will
> > > > > be generated in memory.
> > > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > > baseOffset and
> > > > > startOffset. For example, deleteRecords() could move the
> startOffset
> > > to the
> > > > > middle of a segment. If we copy the full segment to remote
> storage, the
> > > > > baseOffset and the startOffset will be different.
> > > > > 5102.7 Could we define all the public methods for
> > > RemoteLogSegmentMetadata
> > > > > and LogSegmentData?
> > > > > 5102.8 Could we document whether endOffset in
> RemoteLogSegmentMetadata
> > > is
> > > > > inclusive/exclusive?
> > > > >
> > > > > 5103. configs:
> > > > > 5103.1 Could we define the default value of non-required configs
> (e.g
> > > the
> > > > > size of new thread pools)?
> > > > > 5103.2 It seems that local.log.retention.ms should default to
> > > retention.ms,
> > > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > > that local.log.retention.bytes should default to segment.bytes.
> > > > > 5103.3 remote.log.manager.thread.pool.size: The description says
> "used
> > > in
> > > > > scheduling tasks to copy segments, fetch remote log indexes and
> clean
> > > up
> > > > > remote log segments". However, there is a separate
> > > > > config remote.log.reader.threads for fetching remote data. It's
> weird
> > > to
> > > > > fetch remote index and log in different thread pools since both are
> > > used
> > > > > for serving fetch requests.
> > > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of
> > > time to
> > > > > back off when there is no work to do? If so, perhaps it can be
> renamed
> > > as
> > > > > backoff.ms.
> > > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms
> configs?
> > > If
> > > > > so, they need to be listed in this section.
> > > > >
> > > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index
> files
> > > of
> > > > > remote log segments to avoid multiple index fetches from the remote
> > > > > storage." Is the RLM in memory or on disk? If on disk, where is it
> > > stored?
> > > > > Do we need a configuration to bound the size?
> > > > >
> > > > > 5105. The KIP uses local-log-start-offset and Earliest Local
> Offset in
> > > > > different places. It would be useful to standardize the
> terminology.
> > > > >
> > > > > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> > > options
> > > > > without saying which option is chosen.
> > > > >
> > > > > 5107. Follower to leader transition: It has step 2, but not step 1.
> > > > >
> > > > > 5108. If a consumer fetches from the remote data and the remote
> > > storage is
> > > > > not available, what error code is used in the fetch response?
> > > > >
> > > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first
> message
> > > > > offset whose timestamp is >= to the given timestamp in the request.
> > > That
> > > > > means it checks in remote log time indexes first, after which
> local log
> > > > > time indexes are checked." Could you document which method in RLMM
> is
> > > used
> > > > > for this?
> > > > >
> > > > > 5110. Stopreplica: "it sets all the remote log segment metadata of
> that
> > > > > partition with a delete marker and publishes them to RLMM." This
> seems
> > > > > outdated given the new topic deletion logic.
> > > > >
> > > > > 5111. "RLM follower fetches the earliest offset for the earliest
> leader
> > > > > epoch by calling RLMM.earliestLogOffset(TopicPartition
> topicPartition,
> > > int
> > > > > leaderEpoch) and updates that as the log start offset." Do we need
> that
> > > > > since replication propagates logStartOffset already?
> > > > >
> > > > > 5112. Is the default maxWaitMs of 500ms enough for fetching from
> remote
> > > > > storage?
> > > > >
> > > > > 5113. "Committed offsets can be stored in a local file to avoid
> > > reading the
> > > > > messages again when a broker is restarted." Could you describe the
> > > format
> > > > > and the location of the file? Also, could the same message be
> > > processed by
> > > > > RLMM again after broker restart? If so, how do we handle that?
> > > > >
> > > > > 5114. Message format
> > > > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord
> with
> > > > > apiKey 0 and 1.
> > > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> > > endOffset
> > > > > is inclusive/exclusive?
> > > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain
> LeaderEpoch a
> > > bit
> > > > > more? Is that the epoch of the leader when it copies the segment to
> > > remote
> > > > > storage? Also, how will this field be used?
> > > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each
> record
> > > in
> > > > > Kafka already has a timestamp field. Could we just use that?
> > > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > > >
> > > > > 5115. RemoteLogCleaner(RLC): This could be confused with the log
> > > cleaner
> > > > > for compaction. Perhaps it can be renamed to sth like
> > > > > RemotePartitionRemover.
> > > > >
> > > > > 5116. "RLC receives the delete_partition_marked and processes it
> if it
> > > is
> > > > > not yet processed earlier." How does it know whether
> > > > > delete_partition_marked has been processed earlier?
> > > > >
> > > > > 5117. Should we add a new MessageFormatter to read the tier
> metadata
> > > topic?
> > > > >
> > > > > 5118. "Maximum remote log reader thread pool task queue size. If
> the
> > > task
> > > > > queue is full, broker will stop reading remote log segments." What
> do
> > > we
> > > > > return to the fetch request in this case?
> > > > >
> > > > > 5119. It would be useful to list all things not supported in the
> first
> > > > > version in a Future work or Limitations section. For example,
> compacted
> > > > > topic, JBOD, changing remote.log.storage.enable from true to false,
> > > etc.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > > kprakasam@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.duggana@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.duggana@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 <
> > > dhruvil@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.ch@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.duggana@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 (
> > > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > > (
> > > > > > > > > > >>>> jun@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 (
> > > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > > >>>>>>>
> > > > > > >
> > >
> > >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Jun,
Thanks for your comments. Please find the inline replies below.

>605.2 It's rare for the follower to need the remote data. So, the current
approach is fine too. Could you document the process of rebuilding the
producer state since we can't simply trim the producerState to an offset in
the middle of a segment.

Will clarify in the KIP.

>5102.2 Would it be clearer to make startPosiont long and endPosition of
Optional<Long>?

We will have arg checks with respective validation. It is not a good
practice to have arguments with optional as mentioned here.
https://rules.sonarsource.com/java/RSPEC-3553


>5102.5 LogSegmentData still has leaderEpochIndex as File instead of
ByteBuffer.

Updated.

>5102.7 Could you define all public methods for LogSegmentData?

Updated.

>5103.5 Could you change the reference to rlm_process_interval_ms and
rlm_retry_interval_ms to the new config names? Also, the retry interval
config seems still missing. It would be useful to support exponential
backoff with the retry interval config.

Good point. We wanted the retry with truncated exponential backoff,
updated the KIP.

>5111. "RLM follower fetches the earliest offset for the earliest leader
epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
leaderEpoch) and updates that as the log start offset." This text is still
there. Also, could we remove earliestLogOffset() from RLMM?

Updated.

>5115. There are still references to "remote log cleaners".

Updated.

>6000. Since we are returning new error codes, we need to bump up the
protocol version for Fetch request. Also, it will be useful to document all
new error codes and whether they are retriable or not.

Sure, we will add that in the KIP.

>6001. public Map<Long, Long> segmentLeaderEpochs(): Currently, leaderEpoch
is int32 instead of long.

Updated.

>6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
RemoteLogSegmentMetadata.state()?

No, it is fixed.

>6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
topicPartition, long offset, int epochForOffset): Should this return
Optional<RemoteLogSegmentMetadata>?

That makes sense, updated.

>6005. RemoteLogState: It seems it's better to split it between
DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the states
are never shared between the two use cases.

Agree with that, updated.

>6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However, is it
ture that other than the metadata topic, RLMM just needs to know whether
there is a replica assigned to this broker and doesn't need to know whether
the replica is the leader or the follower?

That may be true. If the implementation does not need that, it can
ignore the information in the callback.

>6007: "Handle expired remote segments (leader and follower)": Why is this
needed in both the leader and the follower?

Updated.

>6008.       "name": "SegmentSizeInBytes",
                "type": "int64",
The segment size can just be int32.

Updated.

>6009. For the record format in the log, it seems that we need to add record
type and record version before the serialized bytes. We can follow the
convention used in
https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats

Yes, KIP already mentions that these are serialized before the payload
as below. We will mention explicitly that these two are written before
the data is written.

RLMM instance on broker publishes the message to the topic with key as
null and value with the below format.

type      : unsigned var int, represents the value type. This value is
'apikey' as mentioned in the schema.
version : unsigned var int, the 'version' number of the type as
mentioned in the schema.
data      : record payload in kafka protocol message format.


>6010. remote.log.manager.thread.pool.size: The default value is 10. This
might be too high when enabling the tiered feature for the first time.
Since there are lots of segments that need to be tiered initially, a large
number of threads could overwhelm the broker.

Is the default value 5 reasonable?

6011. "The number of milli seconds to keep the local log segment before it
gets deleted. If not set, the value in `log.retention.minutes` is used. If
set to -1, no time limit is applied." We should use log.retention.ms
instead of log.retention.minutes.
Nice typo catch. Updated the KIP.

Thanks,
Satish.

On Thu, Dec 3, 2020 at 8:03 AM Jun Rao <ju...@confluent.io> wrote:
>
> Hi, Satish,
>
> Thanks for the updated KIP. A few more comments below.
>
> 605.2 It's rare for the follower to need the remote data. So, the current
> approach is fine too. Could you document the process of rebuilding the
> producer state since we can't simply trim the producerState to an offset in
> the middle of a segment.
>
> 5102.2 Would it be clearer to make startPosiont long and endPosition of
> Optional<Long>?
>
> 5102.5 LogSegmentData still has leaderEpochIndex as File instead of
> ByteBuffer.
>
> 5102.7 Could you define all public methods for LogSegmentData?
>
> 5103.5 Could you change the reference to rlm_process_interval_ms and
> rlm_retry_interval_ms to the new config names? Also, the retry interval
> config seems still missing. It would be useful to support exponential
> backoff with the retry interval config.
>
> 5111. "RLM follower fetches the earliest offset for the earliest leader
> epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
> leaderEpoch) and updates that as the log start offset." This text is still
> there. Also, could we remove earliestLogOffset() from RLMM?
>
> 5115. There are still references to "remote log cleaners".
>
> 6000. Since we are returning new error codes, we need to bump up the
> protocol version for Fetch request. Also, it will be useful to document all
> new error codes and whether they are retriable or not.
>
> 6001. public Map<Long, Long> segmentLeaderEpochs(): Currently, leaderEpoch
> is int32 instead of long.
>
> 6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
> RemoteLogSegmentMetadata.state()?
>
> 6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
> topicPartition, long offset, int epochForOffset): Should this return
> Optional<RemoteLogSegmentMetadata>?
>
> 6004. DeletePartitionUpdate.epoch(): It would be useful to pick a more
> indicative name so that people understand what epoch this is.
>
> 6005. RemoteLogState: It seems it's better to split it between
> DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the states
> are never shared between the two use cases.
>
> 6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However, is it
> ture that other than the metadata topic, RLMM just needs to know whether
> there is a replica assigned to this broker and doesn't need to know whether
> the replica is the leader or the follower?
>
> 6007: "Handle expired remote segments (leader and follower)": Why is this
> needed in both the leader and the follower?
>
> 6008.       "name": "SegmentSizeInBytes",
>                 "type": "int64",
> The segment size can just be int32.
>
> 6009. For the record format in the log, it seems that we need to add record
> type and record version before the serialized bytes. We can follow the
> convention used in
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
> .
>
> 6010. remote.log.manager.thread.pool.size: The default value is 10. This
> might be too high when enabling the tiered feature for the first time.
> Since there are lots of segments that need to be tiered initially, a large
> number of threads could overwhelm the broker.
>
> 6011. "The number of milli seconds to keep the local log segment before it
> gets deleted. If not set, the value in `log.retention.minutes` is used. If
> set to -1, no time limit is applied." We should use log.retention.ms
> instead of log.retention.minutes.
>
> Jun
>
> On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <sa...@gmail.com>
> wrote:
>
> > Hi,
> > We updated the KIP with the points mentioned in the earlier mail
> > except for KIP-516 related changes. You can go through them and let us
> > know if you have any comments. We will update the KIP with the
> > remaining todo items and KIP-516 related changes by end of this
> > week(5th Dec).
> >
> > Thanks,
> > Satish.
> >
> > On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <sa...@gmail.com>
> > wrote:
> > >
> > > Hi Jun,
> > > Thanks for your comments. Please find the inline replies below.
> > >
> > > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > > sequence received from remote storage to [LSO, ELO]." I mentioned an
> > issue
> > > earlier. Suppose the leader's local start offset is 100. The follower
> > finds
> > > a remote segment covering offset range [80, 120). The producerState with
> > > this remote segment is up to offset 120. To trim the producerState to
> > > offset 100 requires more work since one needs to download the previous
> > > producerState up to offset 80 and then replay the messages from 80 to
> > 100.
> > > It seems that it's simpler in this case for the follower just to take the
> > > remote segment as it is and start fetching from offset 120.
> > >
> > > We chose that approach to avoid any edge cases here. It may be
> > > possible that the remote log segment that is received may not have the
> > > same leader epoch sequence from 100-120 as it contains on the
> > > leader(this can happen due to unclean leader). It is safe to start
> > > from what the leader returns here.Another way is to find the remote
> > > log segment
> > >
> > > 5016. Just to echo what Kowshik was saying. It seems that
> > > RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> > > partition, not on the replicas for the __remote_log_segment_metadata
> > > partition. It's not clear how the leader of __remote_log_segment_metadata
> > > obtains the metadata for remote segments for deletion.
> > >
> > > RLMM will always receive the callback for the remote log metadata
> > > topic partitions hosted on the local broker and these will be
> > > subscribed. I will make this clear in the KIP.
> > >
> > > 5100. KIP-516 has been accepted and is being implemented now. Could you
> > > update the KIP based on topicID?
> > >
> > > We mentioned KIP-516 and how it helps. We will update this KIP with
> > > all the changes it brings with KIP-516.
> > >
> > > 5101. RLMM: It would be useful to clarify how the following two APIs are
> > > used. According to the wiki, the former is used for topic deletion and
> > the
> > > latter is used for retention. It seems that retention should use the
> > former
> > > since remote segments without a matching epoch in the leader (potentially
> > > due to unclean leader election) also need to be garbage collected. The
> > > latter seems to be used for the new leader to determine the last tiered
> > > segment.
> > >     default Iterator<RemoteLogSegmentMetadata>
> > > listRemoteLogSegments(TopicPartition topicPartition)
> > >     Iterator<RemoteLogSegmentMetadata>
> > listRemoteLogSegments(TopicPartition
> > > topicPartition, long leaderEpoch);
> > >
> > > Right,.that is what we are currently doing. We will update the
> > > javadocs and wiki with that. Earlier, we did not want to remove the
> > > segments which are not matched with leader epochs from the ladder
> > > partition as they may be used later by a replica which can become a
> > > leader (unclean leader election) and refer those segments. But that
> > > may leak these segments in remote storage until the topic lifetime. We
> > > decided to cleanup the segments with the oldest incase of size based
> > > retention also.
> > >
> > > 5102. RSM:
> > > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > >
> > > It will be useful to have metadata for RSM to fetch log segment. It
> > > may create location/path using id with other metadata too.
> > >
> > > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> > >
> > > Wanted to keep endPosition as optional to read till the end of the
> > > segment and avoid sentinels.
> > >
> > > 5102.3 Why only some of the methods have default implementation and
> > others
> > > Don't?
> > >
> > > Actually,  RSM will not have any default implementations. Those 3
> > > methods were made default earlier for tests etc. Updated the wiki.
> > >
> > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > and DeletePartitionUpdate?
> > >
> > > Sure, they will be added.
> > >
> > >
> > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > in leaderEpochIndex as a ByteBuffer or byte array than a file since it
> > will
> > > be generated in memory.
> > >
> > > Right, this is in plan.
> > >
> > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
> > and
> > > startOffset. For example, deleteRecords() could move the startOffset to
> > the
> > > middle of a segment. If we copy the full segment to remote storage, the
> > > baseOffset and the startOffset will be different.
> > >
> > > Good point. startOffset is baseOffset by default, if not set explicitly.
> > >
> > > 5102.7 Could we define all the public methods for
> > RemoteLogSegmentMetadata
> > > and LogSegmentData?
> > >
> > > Sure, updated the wiki.
> > >
> > > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> > > inclusive/exclusive?
> > >
> > > It is inclusive, will update.
> > >
> > > 5103. configs:
> > > 5103.1 Could we define the default value of non-required configs (e.g the
> > > size of new thread pools)?
> > >
> > > Sure, that makes sense.
> > >
> > > 5103.2 It seems that local.log.retention.ms should default to
> > retention.ms,
> > > instead of remote.log.retention.minutes. Similarly, it seems
> > > that local.log.retention.bytes should default to segment.bytes.
> > >
> > > Right, we do not have  remote.log.retention as we discussed earlier.
> > > Thanks for catching the typo.
> > >
> > > 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> > > scheduling tasks to copy segments, fetch remote log indexes and clean up
> > > remote log segments". However, there is a separate
> > > config remote.log.reader.threads for fetching remote data. It's weird to
> > > fetch remote index and log in different thread pools since both are used
> > > for serving fetch requests.
> > >
> > > Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
> > > activities. Fetch path always goes through remote.log.reader.threads.
> > >
> > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time
> > to
> > > back off when there is no work to do? If so, perhaps it can be renamed as
> > > backoff.ms.
> > >
> > > This is the delay interval for each iteration. It may be renamed to
> > > remote.log.manager.task.delay.ms
> > >
> > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> > > so, they need to be listed in this section.
> > >
> > > remote.log.manager.task.interval.ms is the process internal, retry
> > > interval is missing in the configs, which will be updated in the KIP.
> > >
> > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> > > remote log segments to avoid multiple index fetches from the remote
> > > storage." Is the RLM in memory or on disk? If on disk, where is it
> > stored?
> > > Do we need a configuration to bound the size?
> > >
> > > It is stored on disk. They are stored in a directory
> > > `remote-log-index-cache` under log dir. We plan to have a config for
> > > that instead of default. We will have a configuration for that.
> > >
> > > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> > > different places. It would be useful to standardize the terminology.
> > >
> > > Sure.
> > >
> > > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> > options
> > > without saying which option is chosen.
> > > We already mentioned in the KIP that we chose option-2.
> > >
> > > 5107. Follower to leader transition: It has step 2, but not step 1.
> > > Step-1 is there but it is not explicitly highlighted. It is previous
> > > table to step-2.
> > >
> > > 5108. If a consumer fetches from the remote data and the remote storage
> > is
> > > not available, what error code is used in the fetch response?
> > >
> > > Good point. We have not yet defined the error for this case. We need
> > > to define an error message and send the same in fetch response.
> > >
> > > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > > offset whose timestamp is >= to the given timestamp in the request. That
> > > means it checks in remote log time indexes first, after which local log
> > > time indexes are checked." Could you document which method in RLMM is
> > used
> > > for this?
> > >
> > > Okay.
> > >
> > > 5110. Stopreplica: "it sets all the remote log segment metadata of that
> > > partition with a delete marker and publishes them to RLMM." This seems
> > > outdated given the new topic deletion logic.
> > >
> > > Will update with KIP-516 related points.
> > >
> > > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> > int
> > > leaderEpoch) and updates that as the log start offset." Do we need that
> > > since replication propagates logStartOffset already?
> > >
> > > Good point. Right, existing replication protocol takes care of
> > > updating the followers’s log start offset received from the leader.
> > >
> > > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> > > storage?
> > >
> > > Remote reads may fail within the current default wait time, but
> > > subsequent fetches would be able to serve as that data is stored in
> > > the local cache. This cache is currently implemented in RSMs. But we
> > > plan to pull this into the remote log messaging layer in future.
> > >
> > > 5113. "Committed offsets can be stored in a local file to avoid reading
> > the
> > > messages again when a broker is restarted." Could you describe the format
> > > and the location of the file? Also, could the same message be processed
> > by
> > > RLMM again after broker restart? If so, how do we handle that?
> > >
> > > Sure, we will update in the KIP.
> > >
> > > 5114. Message format
> > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> > > apiKey 0 and 1.
> > >
> > > Nice catch, that was a typo. Fixed in the wiki.
> > >
> > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> > endOffset
> > > is inclusive/exclusive?
> > > It is inclusive, will update.
> > >
> > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
> > bit
> > > more? Is that the epoch of the leader when it copies the segment to
> > remote
> > > storage? Also, how will this field be used?
> > >
> > > Right, this is the leader epoch of the broker which copied this
> > > segment. This is helpful in reason about which broker copied the
> > > segment to remote storage.
> > >
> > > 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> > > Kafka already has a timestamp field. Could we just use that?
> > >
> > > This is the  timestamp at which  the respective event occurred. Added
> > > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > > implementation. We thought about that but it looked cleaner to use at
> > > the message structure level instead of getting that from the consumer
> > > record and using that to build the respective event.
> > >
> > >
> > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > >
> > > Right, it looks like config allows only int value >= 14.
> > >
> > > 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> > > for compaction. Perhaps it can be renamed to sth like
> > > RemotePartitionRemover.
> > >
> > > I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we
> > > have other manager classes like RLM, RLMM).
> > >
> > > 5116. "RLC receives the delete_partition_marked and processes it if it is
> > > not yet processed earlier." How does it know whether
> > > delete_partition_marked has been processed earlier?
> > >
> > > This is to handle duplicate delete_partition_marked events. RLC
> > > internally maintains a state for the delete_partition events and if it
> > > already has an existing event then it ignores if it is already being
> > > processed.
> > >
> > > 5117. Should we add a new MessageFormatter to read the tier metadata
> > topic?
> > >
> > > Right, this is in plan but did not mention it in the KIP. This will be
> > > useful for debugging purposes too.
> > >
> > > 5118. "Maximum remote log reader thread pool task queue size. If the task
> > > queue is full, broker will stop reading remote log segments." What do we
> > > return to the fetch request in this case?
> > >
> > > We return an error response for that partition.
> > >
> > > 5119. It would be useful to list all things not supported in the first
> > > version in a Future work or Limitations section. For example, compacted
> > > topic, JBOD, changing remote.log.storage.enable from true to false, etc.
> > >
> > > We already have a non-goals section which is filled with some of these
> > > details. Do we need another limitations section?
> > >
> > > Thanks,
> > > Satish.
> > >
> > > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > Hi, Satish,
> > > >
> > > > Thanks for the updated KIP. A few more comments below.
> > > >
> > > > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > > > sequence received from remote storage to [LSO, ELO]." I mentioned an
> > issue
> > > > earlier. Suppose the leader's local start offset is 100. The follower
> > finds
> > > > a remote segment covering offset range [80, 120). The producerState
> > with
> > > > this remote segment is up to offset 120. To trim the producerState to
> > > > offset 100 requires more work since one needs to download the previous
> > > > producerState up to offset 80 and then replay the messages from 80 to
> > 100.
> > > > It seems that it's simpler in this case for the follower just to take
> > the
> > > > remote segment as it is and start fetching from offset 120.
> > > >
> > > > 5016. Just to echo what Kowshik was saying. It seems that
> > > > RLMM.onPartitionLeadershipChanges() is only called on the replicas for
> > a
> > > > partition, not on the replicas for the __remote_log_segment_metadata
> > > > partition. It's not clear how the leader of
> > __remote_log_segment_metadata
> > > > obtains the metadata for remote segments for deletion.
> > > >
> > > > 5100. KIP-516 has been accepted and is being implemented now. Could you
> > > > update the KIP based on topicID?
> > > >
> > > > 5101. RLMM: It would be useful to clarify how the following two APIs
> > are
> > > > used. According to the wiki, the former is used for topic deletion and
> > the
> > > > latter is used for retention. It seems that retention should use the
> > former
> > > > since remote segments without a matching epoch in the leader
> > (potentially
> > > > due to unclean leader election) also need to be garbage collected. The
> > > > latter seems to be used for the new leader to determine the last tiered
> > > > segment.
> > > >     default Iterator<RemoteLogSegmentMetadata>
> > > > listRemoteLogSegments(TopicPartition topicPartition)
> > > >     Iterator<RemoteLogSegmentMetadata>
> > listRemoteLogSegments(TopicPartition
> > > > topicPartition, long leaderEpoch);
> > > >
> > > > 5102. RSM:
> > > > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> > > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> > > > 5102.3 Why only some of the methods have default implementation and
> > others
> > > > don't?
> > > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > > and DeletePartitionUpdate?
> > > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > > in leaderEpochIndex as a ByteBuffer or byte array than a file since it
> > will
> > > > be generated in memory.
> > > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> > baseOffset and
> > > > startOffset. For example, deleteRecords() could move the startOffset
> > to the
> > > > middle of a segment. If we copy the full segment to remote storage, the
> > > > baseOffset and the startOffset will be different.
> > > > 5102.7 Could we define all the public methods for
> > RemoteLogSegmentMetadata
> > > > and LogSegmentData?
> > > > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata
> > is
> > > > inclusive/exclusive?
> > > >
> > > > 5103. configs:
> > > > 5103.1 Could we define the default value of non-required configs (e.g
> > the
> > > > size of new thread pools)?
> > > > 5103.2 It seems that local.log.retention.ms should default to
> > retention.ms,
> > > > instead of remote.log.retention.minutes. Similarly, it seems
> > > > that local.log.retention.bytes should default to segment.bytes.
> > > > 5103.3 remote.log.manager.thread.pool.size: The description says "used
> > in
> > > > scheduling tasks to copy segments, fetch remote log indexes and clean
> > up
> > > > remote log segments". However, there is a separate
> > > > config remote.log.reader.threads for fetching remote data. It's weird
> > to
> > > > fetch remote index and log in different thread pools since both are
> > used
> > > > for serving fetch requests.
> > > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of
> > time to
> > > > back off when there is no work to do? If so, perhaps it can be renamed
> > as
> > > > backoff.ms.
> > > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs?
> > If
> > > > so, they need to be listed in this section.
> > > >
> > > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files
> > of
> > > > remote log segments to avoid multiple index fetches from the remote
> > > > storage." Is the RLM in memory or on disk? If on disk, where is it
> > stored?
> > > > Do we need a configuration to bound the size?
> > > >
> > > > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> > > > different places. It would be useful to standardize the terminology.
> > > >
> > > > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> > options
> > > > without saying which option is chosen.
> > > >
> > > > 5107. Follower to leader transition: It has step 2, but not step 1.
> > > >
> > > > 5108. If a consumer fetches from the remote data and the remote
> > storage is
> > > > not available, what error code is used in the fetch response?
> > > >
> > > > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > > > offset whose timestamp is >= to the given timestamp in the request.
> > That
> > > > means it checks in remote log time indexes first, after which local log
> > > > time indexes are checked." Could you document which method in RLMM is
> > used
> > > > for this?
> > > >
> > > > 5110. Stopreplica: "it sets all the remote log segment metadata of that
> > > > partition with a delete marker and publishes them to RLMM." This seems
> > > > outdated given the new topic deletion logic.
> > > >
> > > > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > > > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> > int
> > > > leaderEpoch) and updates that as the log start offset." Do we need that
> > > > since replication propagates logStartOffset already?
> > > >
> > > > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> > > > storage?
> > > >
> > > > 5113. "Committed offsets can be stored in a local file to avoid
> > reading the
> > > > messages again when a broker is restarted." Could you describe the
> > format
> > > > and the location of the file? Also, could the same message be
> > processed by
> > > > RLMM again after broker restart? If so, how do we handle that?
> > > >
> > > > 5114. Message format
> > > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> > > > apiKey 0 and 1.
> > > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> > endOffset
> > > > is inclusive/exclusive?
> > > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
> > bit
> > > > more? Is that the epoch of the leader when it copies the segment to
> > remote
> > > > storage? Also, how will this field be used?
> > > > 5114.4 EventTimestamp: Could you explain this a bit more? Each record
> > in
> > > > Kafka already has a timestamp field. Could we just use that?
> > > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > > >
> > > > 5115. RemoteLogCleaner(RLC): This could be confused with the log
> > cleaner
> > > > for compaction. Perhaps it can be renamed to sth like
> > > > RemotePartitionRemover.
> > > >
> > > > 5116. "RLC receives the delete_partition_marked and processes it if it
> > is
> > > > not yet processed earlier." How does it know whether
> > > > delete_partition_marked has been processed earlier?
> > > >
> > > > 5117. Should we add a new MessageFormatter to read the tier metadata
> > topic?
> > > >
> > > > 5118. "Maximum remote log reader thread pool task queue size. If the
> > task
> > > > queue is full, broker will stop reading remote log segments." What do
> > we
> > > > return to the fetch request in this case?
> > > >
> > > > 5119. It would be useful to list all things not supported in the first
> > > > version in a Future work or Limitations section. For example, compacted
> > > > topic, JBOD, changing remote.log.storage.enable from true to false,
> > etc.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> > kprakasam@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.duggana@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.duggana@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 <
> > dhruvil@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 <ha...@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.duggana@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 (
> > > > > > > > > > jun@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.duggana@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.ch@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
> > > > > (
> > > > > > > > > >>>> jun@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 (
> > > > > > > > > >>>>> kafka@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 (
> > > > > > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > > >>>>>>>
> > > > > >
> >
> >

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

Posted by Jun Rao <ju...@confluent.io>.
Hi, Satish,

Thanks for the updated KIP. A few more comments below.

605.2 It's rare for the follower to need the remote data. So, the current
approach is fine too. Could you document the process of rebuilding the
producer state since we can't simply trim the producerState to an offset in
the middle of a segment.

5102.2 Would it be clearer to make startPosiont long and endPosition of
Optional<Long>?

5102.5 LogSegmentData still has leaderEpochIndex as File instead of
ByteBuffer.

5102.7 Could you define all public methods for LogSegmentData?

5103.5 Could you change the reference to rlm_process_interval_ms and
rlm_retry_interval_ms to the new config names? Also, the retry interval
config seems still missing. It would be useful to support exponential
backoff with the retry interval config.

5111. "RLM follower fetches the earliest offset for the earliest leader
epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
leaderEpoch) and updates that as the log start offset." This text is still
there. Also, could we remove earliestLogOffset() from RLMM?

5115. There are still references to "remote log cleaners".

6000. Since we are returning new error codes, we need to bump up the
protocol version for Fetch request. Also, it will be useful to document all
new error codes and whether they are retriable or not.

6001. public Map<Long, Long> segmentLeaderEpochs(): Currently, leaderEpoch
is int32 instead of long.

6002. Is RemoteLogSegmentMetadata.markedForDeletion() needed given
RemoteLogSegmentMetadata.state()?

6003. RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition
topicPartition, long offset, int epochForOffset): Should this return
Optional<RemoteLogSegmentMetadata>?

6004. DeletePartitionUpdate.epoch(): It would be useful to pick a more
indicative name so that people understand what epoch this is.

6005. RemoteLogState: It seems it's better to split it between
DeletePartitionUpdate and RemoteLogSegmentMetadataUpdate since the states
are never shared between the two use cases.

6006. RLMM.onPartitionLeadershipChanges(): This may be ok. However, is it
ture that other than the metadata topic, RLMM just needs to know whether
there is a replica assigned to this broker and doesn't need to know whether
the replica is the leader or the follower?

6007: "Handle expired remote segments (leader and follower)": Why is this
needed in both the leader and the follower?

6008.       "name": "SegmentSizeInBytes",
                "type": "int64",
The segment size can just be int32.

6009. For the record format in the log, it seems that we need to add record
type and record version before the serialized bytes. We can follow the
convention used in
https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorum-based+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-RecordFormats
.

6010. remote.log.manager.thread.pool.size: The default value is 10. This
might be too high when enabling the tiered feature for the first time.
Since there are lots of segments that need to be tiered initially, a large
number of threads could overwhelm the broker.

6011. "The number of milli seconds to keep the local log segment before it
gets deleted. If not set, the value in `log.retention.minutes` is used. If
set to -1, no time limit is applied." We should use log.retention.ms
instead of log.retention.minutes.

Jun

On Tue, Dec 1, 2020 at 2:42 AM Satish Duggana <sa...@gmail.com>
wrote:

> Hi,
> We updated the KIP with the points mentioned in the earlier mail
> except for KIP-516 related changes. You can go through them and let us
> know if you have any comments. We will update the KIP with the
> remaining todo items and KIP-516 related changes by end of this
> week(5th Dec).
>
> Thanks,
> Satish.
>
> On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <sa...@gmail.com>
> wrote:
> >
> > Hi Jun,
> > Thanks for your comments. Please find the inline replies below.
> >
> > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > sequence received from remote storage to [LSO, ELO]." I mentioned an
> issue
> > earlier. Suppose the leader's local start offset is 100. The follower
> finds
> > a remote segment covering offset range [80, 120). The producerState with
> > this remote segment is up to offset 120. To trim the producerState to
> > offset 100 requires more work since one needs to download the previous
> > producerState up to offset 80 and then replay the messages from 80 to
> 100.
> > It seems that it's simpler in this case for the follower just to take the
> > remote segment as it is and start fetching from offset 120.
> >
> > We chose that approach to avoid any edge cases here. It may be
> > possible that the remote log segment that is received may not have the
> > same leader epoch sequence from 100-120 as it contains on the
> > leader(this can happen due to unclean leader). It is safe to start
> > from what the leader returns here.Another way is to find the remote
> > log segment
> >
> > 5016. Just to echo what Kowshik was saying. It seems that
> > RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> > partition, not on the replicas for the __remote_log_segment_metadata
> > partition. It's not clear how the leader of __remote_log_segment_metadata
> > obtains the metadata for remote segments for deletion.
> >
> > RLMM will always receive the callback for the remote log metadata
> > topic partitions hosted on the local broker and these will be
> > subscribed. I will make this clear in the KIP.
> >
> > 5100. KIP-516 has been accepted and is being implemented now. Could you
> > update the KIP based on topicID?
> >
> > We mentioned KIP-516 and how it helps. We will update this KIP with
> > all the changes it brings with KIP-516.
> >
> > 5101. RLMM: It would be useful to clarify how the following two APIs are
> > used. According to the wiki, the former is used for topic deletion and
> the
> > latter is used for retention. It seems that retention should use the
> former
> > since remote segments without a matching epoch in the leader (potentially
> > due to unclean leader election) also need to be garbage collected. The
> > latter seems to be used for the new leader to determine the last tiered
> > segment.
> >     default Iterator<RemoteLogSegmentMetadata>
> > listRemoteLogSegments(TopicPartition topicPartition)
> >     Iterator<RemoteLogSegmentMetadata>
> listRemoteLogSegments(TopicPartition
> > topicPartition, long leaderEpoch);
> >
> > Right,.that is what we are currently doing. We will update the
> > javadocs and wiki with that. Earlier, we did not want to remove the
> > segments which are not matched with leader epochs from the ladder
> > partition as they may be used later by a replica which can become a
> > leader (unclean leader election) and refer those segments. But that
> > may leak these segments in remote storage until the topic lifetime. We
> > decided to cleanup the segments with the oldest incase of size based
> > retention also.
> >
> > 5102. RSM:
> > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> >
> > It will be useful to have metadata for RSM to fetch log segment. It
> > may create location/path using id with other metadata too.
> >
> > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> >
> > Wanted to keep endPosition as optional to read till the end of the
> > segment and avoid sentinels.
> >
> > 5102.3 Why only some of the methods have default implementation and
> others
> > Don't?
> >
> > Actually,  RSM will not have any default implementations. Those 3
> > methods were made default earlier for tests etc. Updated the wiki.
> >
> > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > and DeletePartitionUpdate?
> >
> > Sure, they will be added.
> >
> >
> > 5102.5 LogSegmentData: It seems that it's easier to pass
> > in leaderEpochIndex as a ByteBuffer or byte array than a file since it
> will
> > be generated in memory.
> >
> > Right, this is in plan.
> >
> > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset
> and
> > startOffset. For example, deleteRecords() could move the startOffset to
> the
> > middle of a segment. If we copy the full segment to remote storage, the
> > baseOffset and the startOffset will be different.
> >
> > Good point. startOffset is baseOffset by default, if not set explicitly.
> >
> > 5102.7 Could we define all the public methods for
> RemoteLogSegmentMetadata
> > and LogSegmentData?
> >
> > Sure, updated the wiki.
> >
> > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> > inclusive/exclusive?
> >
> > It is inclusive, will update.
> >
> > 5103. configs:
> > 5103.1 Could we define the default value of non-required configs (e.g the
> > size of new thread pools)?
> >
> > Sure, that makes sense.
> >
> > 5103.2 It seems that local.log.retention.ms should default to
> retention.ms,
> > instead of remote.log.retention.minutes. Similarly, it seems
> > that local.log.retention.bytes should default to segment.bytes.
> >
> > Right, we do not have  remote.log.retention as we discussed earlier.
> > Thanks for catching the typo.
> >
> > 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> > scheduling tasks to copy segments, fetch remote log indexes and clean up
> > remote log segments". However, there is a separate
> > config remote.log.reader.threads for fetching remote data. It's weird to
> > fetch remote index and log in different thread pools since both are used
> > for serving fetch requests.
> >
> > Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
> > activities. Fetch path always goes through remote.log.reader.threads.
> >
> > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time
> to
> > back off when there is no work to do? If so, perhaps it can be renamed as
> > backoff.ms.
> >
> > This is the delay interval for each iteration. It may be renamed to
> > remote.log.manager.task.delay.ms
> >
> > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> > so, they need to be listed in this section.
> >
> > remote.log.manager.task.interval.ms is the process internal, retry
> > interval is missing in the configs, which will be updated in the KIP.
> >
> > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> > remote log segments to avoid multiple index fetches from the remote
> > storage." Is the RLM in memory or on disk? If on disk, where is it
> stored?
> > Do we need a configuration to bound the size?
> >
> > It is stored on disk. They are stored in a directory
> > `remote-log-index-cache` under log dir. We plan to have a config for
> > that instead of default. We will have a configuration for that.
> >
> > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> > different places. It would be useful to standardize the terminology.
> >
> > Sure.
> >
> > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> options
> > without saying which option is chosen.
> > We already mentioned in the KIP that we chose option-2.
> >
> > 5107. Follower to leader transition: It has step 2, but not step 1.
> > Step-1 is there but it is not explicitly highlighted. It is previous
> > table to step-2.
> >
> > 5108. If a consumer fetches from the remote data and the remote storage
> is
> > not available, what error code is used in the fetch response?
> >
> > Good point. We have not yet defined the error for this case. We need
> > to define an error message and send the same in fetch response.
> >
> > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > offset whose timestamp is >= to the given timestamp in the request. That
> > means it checks in remote log time indexes first, after which local log
> > time indexes are checked." Could you document which method in RLMM is
> used
> > for this?
> >
> > Okay.
> >
> > 5110. Stopreplica: "it sets all the remote log segment metadata of that
> > partition with a delete marker and publishes them to RLMM." This seems
> > outdated given the new topic deletion logic.
> >
> > Will update with KIP-516 related points.
> >
> > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> int
> > leaderEpoch) and updates that as the log start offset." Do we need that
> > since replication propagates logStartOffset already?
> >
> > Good point. Right, existing replication protocol takes care of
> > updating the followers’s log start offset received from the leader.
> >
> > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> > storage?
> >
> > Remote reads may fail within the current default wait time, but
> > subsequent fetches would be able to serve as that data is stored in
> > the local cache. This cache is currently implemented in RSMs. But we
> > plan to pull this into the remote log messaging layer in future.
> >
> > 5113. "Committed offsets can be stored in a local file to avoid reading
> the
> > messages again when a broker is restarted." Could you describe the format
> > and the location of the file? Also, could the same message be processed
> by
> > RLMM again after broker restart? If so, how do we handle that?
> >
> > Sure, we will update in the KIP.
> >
> > 5114. Message format
> > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> > apiKey 0 and 1.
> >
> > Nice catch, that was a typo. Fixed in the wiki.
> >
> > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> endOffset
> > is inclusive/exclusive?
> > It is inclusive, will update.
> >
> > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
> bit
> > more? Is that the epoch of the leader when it copies the segment to
> remote
> > storage? Also, how will this field be used?
> >
> > Right, this is the leader epoch of the broker which copied this
> > segment. This is helpful in reason about which broker copied the
> > segment to remote storage.
> >
> > 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> > Kafka already has a timestamp field. Could we just use that?
> >
> > This is the  timestamp at which  the respective event occurred. Added
> > this  to RemoteLogSegmentMetadata as RLMM can be  any other
> > implementation. We thought about that but it looked cleaner to use at
> > the message structure level instead of getting that from the consumer
> > record and using that to build the respective event.
> >
> >
> > 5114.5 SegmentSizeInBytes: Could this just be int32?
> >
> > Right, it looks like config allows only int value >= 14.
> >
> > 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> > for compaction. Perhaps it can be renamed to sth like
> > RemotePartitionRemover.
> >
> > I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we
> > have other manager classes like RLM, RLMM).
> >
> > 5116. "RLC receives the delete_partition_marked and processes it if it is
> > not yet processed earlier." How does it know whether
> > delete_partition_marked has been processed earlier?
> >
> > This is to handle duplicate delete_partition_marked events. RLC
> > internally maintains a state for the delete_partition events and if it
> > already has an existing event then it ignores if it is already being
> > processed.
> >
> > 5117. Should we add a new MessageFormatter to read the tier metadata
> topic?
> >
> > Right, this is in plan but did not mention it in the KIP. This will be
> > useful for debugging purposes too.
> >
> > 5118. "Maximum remote log reader thread pool task queue size. If the task
> > queue is full, broker will stop reading remote log segments." What do we
> > return to the fetch request in this case?
> >
> > We return an error response for that partition.
> >
> > 5119. It would be useful to list all things not supported in the first
> > version in a Future work or Limitations section. For example, compacted
> > topic, JBOD, changing remote.log.storage.enable from true to false, etc.
> >
> > We already have a non-goals section which is filled with some of these
> > details. Do we need another limitations section?
> >
> > Thanks,
> > Satish.
> >
> > On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > Hi, Satish,
> > >
> > > Thanks for the updated KIP. A few more comments below.
> > >
> > > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > > sequence received from remote storage to [LSO, ELO]." I mentioned an
> issue
> > > earlier. Suppose the leader's local start offset is 100. The follower
> finds
> > > a remote segment covering offset range [80, 120). The producerState
> with
> > > this remote segment is up to offset 120. To trim the producerState to
> > > offset 100 requires more work since one needs to download the previous
> > > producerState up to offset 80 and then replay the messages from 80 to
> 100.
> > > It seems that it's simpler in this case for the follower just to take
> the
> > > remote segment as it is and start fetching from offset 120.
> > >
> > > 5016. Just to echo what Kowshik was saying. It seems that
> > > RLMM.onPartitionLeadershipChanges() is only called on the replicas for
> a
> > > partition, not on the replicas for the __remote_log_segment_metadata
> > > partition. It's not clear how the leader of
> __remote_log_segment_metadata
> > > obtains the metadata for remote segments for deletion.
> > >
> > > 5100. KIP-516 has been accepted and is being implemented now. Could you
> > > update the KIP based on topicID?
> > >
> > > 5101. RLMM: It would be useful to clarify how the following two APIs
> are
> > > used. According to the wiki, the former is used for topic deletion and
> the
> > > latter is used for retention. It seems that retention should use the
> former
> > > since remote segments without a matching epoch in the leader
> (potentially
> > > due to unclean leader election) also need to be garbage collected. The
> > > latter seems to be used for the new leader to determine the last tiered
> > > segment.
> > >     default Iterator<RemoteLogSegmentMetadata>
> > > listRemoteLogSegments(TopicPartition topicPartition)
> > >     Iterator<RemoteLogSegmentMetadata>
> listRemoteLogSegments(TopicPartition
> > > topicPartition, long leaderEpoch);
> > >
> > > 5102. RSM:
> > > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> > > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> > > 5102.3 Why only some of the methods have default implementation and
> others
> > > don't?
> > > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > > and DeletePartitionUpdate?
> > > 5102.5 LogSegmentData: It seems that it's easier to pass
> > > in leaderEpochIndex as a ByteBuffer or byte array than a file since it
> will
> > > be generated in memory.
> > > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both
> baseOffset and
> > > startOffset. For example, deleteRecords() could move the startOffset
> to the
> > > middle of a segment. If we copy the full segment to remote storage, the
> > > baseOffset and the startOffset will be different.
> > > 5102.7 Could we define all the public methods for
> RemoteLogSegmentMetadata
> > > and LogSegmentData?
> > > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata
> is
> > > inclusive/exclusive?
> > >
> > > 5103. configs:
> > > 5103.1 Could we define the default value of non-required configs (e.g
> the
> > > size of new thread pools)?
> > > 5103.2 It seems that local.log.retention.ms should default to
> retention.ms,
> > > instead of remote.log.retention.minutes. Similarly, it seems
> > > that local.log.retention.bytes should default to segment.bytes.
> > > 5103.3 remote.log.manager.thread.pool.size: The description says "used
> in
> > > scheduling tasks to copy segments, fetch remote log indexes and clean
> up
> > > remote log segments". However, there is a separate
> > > config remote.log.reader.threads for fetching remote data. It's weird
> to
> > > fetch remote index and log in different thread pools since both are
> used
> > > for serving fetch requests.
> > > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of
> time to
> > > back off when there is no work to do? If so, perhaps it can be renamed
> as
> > > backoff.ms.
> > > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs?
> If
> > > so, they need to be listed in this section.
> > >
> > > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files
> of
> > > remote log segments to avoid multiple index fetches from the remote
> > > storage." Is the RLM in memory or on disk? If on disk, where is it
> stored?
> > > Do we need a configuration to bound the size?
> > >
> > > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> > > different places. It would be useful to standardize the terminology.
> > >
> > > 5106. The section on "In BuildingRemoteLogAux state". It listed two
> options
> > > without saying which option is chosen.
> > >
> > > 5107. Follower to leader transition: It has step 2, but not step 1.
> > >
> > > 5108. If a consumer fetches from the remote data and the remote
> storage is
> > > not available, what error code is used in the fetch response?
> > >
> > > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > > offset whose timestamp is >= to the given timestamp in the request.
> That
> > > means it checks in remote log time indexes first, after which local log
> > > time indexes are checked." Could you document which method in RLMM is
> used
> > > for this?
> > >
> > > 5110. Stopreplica: "it sets all the remote log segment metadata of that
> > > partition with a delete marker and publishes them to RLMM." This seems
> > > outdated given the new topic deletion logic.
> > >
> > > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition,
> int
> > > leaderEpoch) and updates that as the log start offset." Do we need that
> > > since replication propagates logStartOffset already?
> > >
> > > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> > > storage?
> > >
> > > 5113. "Committed offsets can be stored in a local file to avoid
> reading the
> > > messages again when a broker is restarted." Could you describe the
> format
> > > and the location of the file? Also, could the same message be
> processed by
> > > RLMM again after broker restart? If so, how do we handle that?
> > >
> > > 5114. Message format
> > > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> > > apiKey 0 and 1.
> > > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether
> endOffset
> > > is inclusive/exclusive?
> > > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a
> bit
> > > more? Is that the epoch of the leader when it copies the segment to
> remote
> > > storage? Also, how will this field be used?
> > > 5114.4 EventTimestamp: Could you explain this a bit more? Each record
> in
> > > Kafka already has a timestamp field. Could we just use that?
> > > 5114.5 SegmentSizeInBytes: Could this just be int32?
> > >
> > > 5115. RemoteLogCleaner(RLC): This could be confused with the log
> cleaner
> > > for compaction. Perhaps it can be renamed to sth like
> > > RemotePartitionRemover.
> > >
> > > 5116. "RLC receives the delete_partition_marked and processes it if it
> is
> > > not yet processed earlier." How does it know whether
> > > delete_partition_marked has been processed earlier?
> > >
> > > 5117. Should we add a new MessageFormatter to read the tier metadata
> topic?
> > >
> > > 5118. "Maximum remote log reader thread pool task queue size. If the
> task
> > > queue is full, broker will stop reading remote log segments." What do
> we
> > > return to the fetch request in this case?
> > >
> > > 5119. It would be useful to list all things not supported in the first
> > > version in a Future work or Limitations section. For example, compacted
> > > topic, JBOD, changing remote.log.storage.enable from true to false,
> etc.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <
> kprakasam@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.duggana@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.duggana@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 <
> dhruvil@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 <ha...@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.duggana@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 (
> > > > > > > > > jun@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.duggana@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.ch@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
> > > > (
> > > > > > > > >>>> jun@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 (
> > > > > > > > >>>>> kafka@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 (
> > > > > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > > >>>>>>>
> > > > >
>
>

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

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

Thanks,
Satish.

On Tue, Nov 10, 2020 at 8:26 PM Satish Duggana <sa...@gmail.com> wrote:
>
> Hi Jun,
> Thanks for your comments. Please find the inline replies below.
>
> 605.2 "Build the local leader epoch cache by cutting the leader epoch
> sequence received from remote storage to [LSO, ELO]." I mentioned an issue
> earlier. Suppose the leader's local start offset is 100. The follower finds
> a remote segment covering offset range [80, 120). The producerState with
> this remote segment is up to offset 120. To trim the producerState to
> offset 100 requires more work since one needs to download the previous
> producerState up to offset 80 and then replay the messages from 80 to 100.
> It seems that it's simpler in this case for the follower just to take the
> remote segment as it is and start fetching from offset 120.
>
> We chose that approach to avoid any edge cases here. It may be
> possible that the remote log segment that is received may not have the
> same leader epoch sequence from 100-120 as it contains on the
> leader(this can happen due to unclean leader). It is safe to start
> from what the leader returns here.Another way is to find the remote
> log segment
>
> 5016. Just to echo what Kowshik was saying. It seems that
> RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> partition, not on the replicas for the __remote_log_segment_metadata
> partition. It's not clear how the leader of __remote_log_segment_metadata
> obtains the metadata for remote segments for deletion.
>
> RLMM will always receive the callback for the remote log metadata
> topic partitions hosted on the local broker and these will be
> subscribed. I will make this clear in the KIP.
>
> 5100. KIP-516 has been accepted and is being implemented now. Could you
> update the KIP based on topicID?
>
> We mentioned KIP-516 and how it helps. We will update this KIP with
> all the changes it brings with KIP-516.
>
> 5101. RLMM: It would be useful to clarify how the following two APIs are
> used. According to the wiki, the former is used for topic deletion and the
> latter is used for retention. It seems that retention should use the former
> since remote segments without a matching epoch in the leader (potentially
> due to unclean leader election) also need to be garbage collected. The
> latter seems to be used for the new leader to determine the last tiered
> segment.
>     default Iterator<RemoteLogSegmentMetadata>
> listRemoteLogSegments(TopicPartition topicPartition)
>     Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
> topicPartition, long leaderEpoch);
>
> Right,.that is what we are currently doing. We will update the
> javadocs and wiki with that. Earlier, we did not want to remove the
> segments which are not matched with leader epochs from the ladder
> partition as they may be used later by a replica which can become a
> leader (unclean leader election) and refer those segments. But that
> may leak these segments in remote storage until the topic lifetime. We
> decided to cleanup the segments with the oldest incase of size based
> retention also.
>
> 5102. RSM:
> 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
>
> It will be useful to have metadata for RSM to fetch log segment. It
> may create location/path using id with other metadata too.
>
> 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
>
> Wanted to keep endPosition as optional to read till the end of the
> segment and avoid sentinels.
>
> 5102.3 Why only some of the methods have default implementation and others
> Don't?
>
> Actually,  RSM will not have any default implementations. Those 3
> methods were made default earlier for tests etc. Updated the wiki.
>
> 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> and DeletePartitionUpdate?
>
> Sure, they will be added.
>
>
> 5102.5 LogSegmentData: It seems that it's easier to pass
> in leaderEpochIndex as a ByteBuffer or byte array than a file since it will
> be generated in memory.
>
> Right, this is in plan.
>
> 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset and
> startOffset. For example, deleteRecords() could move the startOffset to the
> middle of a segment. If we copy the full segment to remote storage, the
> baseOffset and the startOffset will be different.
>
> Good point. startOffset is baseOffset by default, if not set explicitly.
>
> 5102.7 Could we define all the public methods for RemoteLogSegmentMetadata
> and LogSegmentData?
>
> Sure, updated the wiki.
>
> 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> inclusive/exclusive?
>
> It is inclusive, will update.
>
> 5103. configs:
> 5103.1 Could we define the default value of non-required configs (e.g the
> size of new thread pools)?
>
> Sure, that makes sense.
>
> 5103.2 It seems that local.log.retention.ms should default to retention.ms,
> instead of remote.log.retention.minutes. Similarly, it seems
> that local.log.retention.bytes should default to segment.bytes.
>
> Right, we do not have  remote.log.retention as we discussed earlier.
> Thanks for catching the typo.
>
> 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> scheduling tasks to copy segments, fetch remote log indexes and clean up
> remote log segments". However, there is a separate
> config remote.log.reader.threads for fetching remote data. It's weird to
> fetch remote index and log in different thread pools since both are used
> for serving fetch requests.
>
> Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
> activities. Fetch path always goes through remote.log.reader.threads.
>
> 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time to
> back off when there is no work to do? If so, perhaps it can be renamed as
> backoff.ms.
>
> This is the delay interval for each iteration. It may be renamed to
> remote.log.manager.task.delay.ms
>
> 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> so, they need to be listed in this section.
>
> remote.log.manager.task.interval.ms is the process internal, retry
> interval is missing in the configs, which will be updated in the KIP.
>
> 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> remote log segments to avoid multiple index fetches from the remote
> storage." Is the RLM in memory or on disk? If on disk, where is it stored?
> Do we need a configuration to bound the size?
>
> It is stored on disk. They are stored in a directory
> `remote-log-index-cache` under log dir. We plan to have a config for
> that instead of default. We will have a configuration for that.
>
> 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> different places. It would be useful to standardize the terminology.
>
> Sure.
>
> 5106. The section on "In BuildingRemoteLogAux state". It listed two options
> without saying which option is chosen.
> We already mentioned in the KIP that we chose option-2.
>
> 5107. Follower to leader transition: It has step 2, but not step 1.
> Step-1 is there but it is not explicitly highlighted. It is previous
> table to step-2.
>
> 5108. If a consumer fetches from the remote data and the remote storage is
> not available, what error code is used in the fetch response?
>
> Good point. We have not yet defined the error for this case. We need
> to define an error message and send the same in fetch response.
>
> 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> offset whose timestamp is >= to the given timestamp in the request. That
> means it checks in remote log time indexes first, after which local log
> time indexes are checked." Could you document which method in RLMM is used
> for this?
>
> Okay.
>
> 5110. Stopreplica: "it sets all the remote log segment metadata of that
> partition with a delete marker and publishes them to RLMM." This seems
> outdated given the new topic deletion logic.
>
> Will update with KIP-516 related points.
>
> 5111. "RLM follower fetches the earliest offset for the earliest leader
> epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
> leaderEpoch) and updates that as the log start offset." Do we need that
> since replication propagates logStartOffset already?
>
> Good point. Right, existing replication protocol takes care of
> updating the followers’s log start offset received from the leader.
>
> 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> storage?
>
> Remote reads may fail within the current default wait time, but
> subsequent fetches would be able to serve as that data is stored in
> the local cache. This cache is currently implemented in RSMs. But we
> plan to pull this into the remote log messaging layer in future.
>
> 5113. "Committed offsets can be stored in a local file to avoid reading the
> messages again when a broker is restarted." Could you describe the format
> and the location of the file? Also, could the same message be processed by
> RLMM again after broker restart? If so, how do we handle that?
>
> Sure, we will update in the KIP.
>
> 5114. Message format
> 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> apiKey 0 and 1.
>
> Nice catch, that was a typo. Fixed in the wiki.
>
> 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether endOffset
> is inclusive/exclusive?
> It is inclusive, will update.
>
> 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a bit
> more? Is that the epoch of the leader when it copies the segment to remote
> storage? Also, how will this field be used?
>
> Right, this is the leader epoch of the broker which copied this
> segment. This is helpful in reason about which broker copied the
> segment to remote storage.
>
> 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> Kafka already has a timestamp field. Could we just use that?
>
> This is the  timestamp at which  the respective event occurred. Added
> this  to RemoteLogSegmentMetadata as RLMM can be  any other
> implementation. We thought about that but it looked cleaner to use at
> the message structure level instead of getting that from the consumer
> record and using that to build the respective event.
>
>
> 5114.5 SegmentSizeInBytes: Could this just be int32?
>
> Right, it looks like config allows only int value >= 14.
>
> 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> for compaction. Perhaps it can be renamed to sth like
> RemotePartitionRemover.
>
> I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we
> have other manager classes like RLM, RLMM).
>
> 5116. "RLC receives the delete_partition_marked and processes it if it is
> not yet processed earlier." How does it know whether
> delete_partition_marked has been processed earlier?
>
> This is to handle duplicate delete_partition_marked events. RLC
> internally maintains a state for the delete_partition events and if it
> already has an existing event then it ignores if it is already being
> processed.
>
> 5117. Should we add a new MessageFormatter to read the tier metadata topic?
>
> Right, this is in plan but did not mention it in the KIP. This will be
> useful for debugging purposes too.
>
> 5118. "Maximum remote log reader thread pool task queue size. If the task
> queue is full, broker will stop reading remote log segments." What do we
> return to the fetch request in this case?
>
> We return an error response for that partition.
>
> 5119. It would be useful to list all things not supported in the first
> version in a Future work or Limitations section. For example, compacted
> topic, JBOD, changing remote.log.storage.enable from true to false, etc.
>
> We already have a non-goals section which is filled with some of these
> details. Do we need another limitations section?
>
> Thanks,
> Satish.
>
> On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
> >
> > Hi, Satish,
> >
> > Thanks for the updated KIP. A few more comments below.
> >
> > 605.2 "Build the local leader epoch cache by cutting the leader epoch
> > sequence received from remote storage to [LSO, ELO]." I mentioned an issue
> > earlier. Suppose the leader's local start offset is 100. The follower finds
> > a remote segment covering offset range [80, 120). The producerState with
> > this remote segment is up to offset 120. To trim the producerState to
> > offset 100 requires more work since one needs to download the previous
> > producerState up to offset 80 and then replay the messages from 80 to 100.
> > It seems that it's simpler in this case for the follower just to take the
> > remote segment as it is and start fetching from offset 120.
> >
> > 5016. Just to echo what Kowshik was saying. It seems that
> > RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> > partition, not on the replicas for the __remote_log_segment_metadata
> > partition. It's not clear how the leader of __remote_log_segment_metadata
> > obtains the metadata for remote segments for deletion.
> >
> > 5100. KIP-516 has been accepted and is being implemented now. Could you
> > update the KIP based on topicID?
> >
> > 5101. RLMM: It would be useful to clarify how the following two APIs are
> > used. According to the wiki, the former is used for topic deletion and the
> > latter is used for retention. It seems that retention should use the former
> > since remote segments without a matching epoch in the leader (potentially
> > due to unclean leader election) also need to be garbage collected. The
> > latter seems to be used for the new leader to determine the last tiered
> > segment.
> >     default Iterator<RemoteLogSegmentMetadata>
> > listRemoteLogSegments(TopicPartition topicPartition)
> >     Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
> > topicPartition, long leaderEpoch);
> >
> > 5102. RSM:
> > 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> > use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> > 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> > 5102.3 Why only some of the methods have default implementation and others
> > don't?
> > 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> > and DeletePartitionUpdate?
> > 5102.5 LogSegmentData: It seems that it's easier to pass
> > in leaderEpochIndex as a ByteBuffer or byte array than a file since it will
> > be generated in memory.
> > 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset and
> > startOffset. For example, deleteRecords() could move the startOffset to the
> > middle of a segment. If we copy the full segment to remote storage, the
> > baseOffset and the startOffset will be different.
> > 5102.7 Could we define all the public methods for RemoteLogSegmentMetadata
> > and LogSegmentData?
> > 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> > inclusive/exclusive?
> >
> > 5103. configs:
> > 5103.1 Could we define the default value of non-required configs (e.g the
> > size of new thread pools)?
> > 5103.2 It seems that local.log.retention.ms should default to retention.ms,
> > instead of remote.log.retention.minutes. Similarly, it seems
> > that local.log.retention.bytes should default to segment.bytes.
> > 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> > scheduling tasks to copy segments, fetch remote log indexes and clean up
> > remote log segments". However, there is a separate
> > config remote.log.reader.threads for fetching remote data. It's weird to
> > fetch remote index and log in different thread pools since both are used
> > for serving fetch requests.
> > 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time to
> > back off when there is no work to do? If so, perhaps it can be renamed as
> > backoff.ms.
> > 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> > so, they need to be listed in this section.
> >
> > 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> > remote log segments to avoid multiple index fetches from the remote
> > storage." Is the RLM in memory or on disk? If on disk, where is it stored?
> > Do we need a configuration to bound the size?
> >
> > 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> > different places. It would be useful to standardize the terminology.
> >
> > 5106. The section on "In BuildingRemoteLogAux state". It listed two options
> > without saying which option is chosen.
> >
> > 5107. Follower to leader transition: It has step 2, but not step 1.
> >
> > 5108. If a consumer fetches from the remote data and the remote storage is
> > not available, what error code is used in the fetch response?
> >
> > 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> > offset whose timestamp is >= to the given timestamp in the request. That
> > means it checks in remote log time indexes first, after which local log
> > time indexes are checked." Could you document which method in RLMM is used
> > for this?
> >
> > 5110. Stopreplica: "it sets all the remote log segment metadata of that
> > partition with a delete marker and publishes them to RLMM." This seems
> > outdated given the new topic deletion logic.
> >
> > 5111. "RLM follower fetches the earliest offset for the earliest leader
> > epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
> > leaderEpoch) and updates that as the log start offset." Do we need that
> > since replication propagates logStartOffset already?
> >
> > 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> > storage?
> >
> > 5113. "Committed offsets can be stored in a local file to avoid reading the
> > messages again when a broker is restarted." Could you describe the format
> > and the location of the file? Also, could the same message be processed by
> > RLMM again after broker restart? If so, how do we handle that?
> >
> > 5114. Message format
> > 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> > apiKey 0 and 1.
> > 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether endOffset
> > is inclusive/exclusive?
> > 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a bit
> > more? Is that the epoch of the leader when it copies the segment to remote
> > storage? Also, how will this field be used?
> > 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> > Kafka already has a timestamp field. Could we just use that?
> > 5114.5 SegmentSizeInBytes: Could this just be int32?
> >
> > 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> > for compaction. Perhaps it can be renamed to sth like
> > RemotePartitionRemover.
> >
> > 5116. "RLC receives the delete_partition_marked and processes it if it is
> > not yet processed earlier." How does it know whether
> > delete_partition_marked has been processed earlier?
> >
> > 5117. Should we add a new MessageFormatter to read the tier metadata topic?
> >
> > 5118. "Maximum remote log reader thread pool task queue size. If the task
> > queue is full, broker will stop reading remote log segments." What do we
> > return to the fetch request in this case?
> >
> > 5119. It would be useful to list all things not supported in the first
> > version in a Future work or Limitations section. For example, compacted
> > topic, JBOD, changing remote.log.storage.enable from true to false, etc.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <kp...@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 <sa...@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.duggana@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 <dh...@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 <ha...@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.duggana@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 (
> > > > > > > > jun@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.duggana@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.ch@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
> > > (
> > > > > > > >>>> jun@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 (
> > > > > > > >>>>> kafka@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 (
> > > > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > > >>>>>>>
> > > >


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

Posted by Satish Duggana <sa...@gmail.com>.
Hi Jun,
Thanks for your comments. Please find the inline replies below.

605.2 "Build the local leader epoch cache by cutting the leader epoch
sequence received from remote storage to [LSO, ELO]." I mentioned an issue
earlier. Suppose the leader's local start offset is 100. The follower finds
a remote segment covering offset range [80, 120). The producerState with
this remote segment is up to offset 120. To trim the producerState to
offset 100 requires more work since one needs to download the previous
producerState up to offset 80 and then replay the messages from 80 to 100.
It seems that it's simpler in this case for the follower just to take the
remote segment as it is and start fetching from offset 120.

We chose that approach to avoid any edge cases here. It may be
possible that the remote log segment that is received may not have the
same leader epoch sequence from 100-120 as it contains on the
leader(this can happen due to unclean leader). It is safe to start
from what the leader returns here.Another way is to find the remote
log segment

5016. Just to echo what Kowshik was saying. It seems that
RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
partition, not on the replicas for the __remote_log_segment_metadata
partition. It's not clear how the leader of __remote_log_segment_metadata
obtains the metadata for remote segments for deletion.

RLMM will always receive the callback for the remote log metadata
topic partitions hosted on the local broker and these will be
subscribed. I will make this clear in the KIP.

5100. KIP-516 has been accepted and is being implemented now. Could you
update the KIP based on topicID?

We mentioned KIP-516 and how it helps. We will update this KIP with
all the changes it brings with KIP-516.

5101. RLMM: It would be useful to clarify how the following two APIs are
used. According to the wiki, the former is used for topic deletion and the
latter is used for retention. It seems that retention should use the former
since remote segments without a matching epoch in the leader (potentially
due to unclean leader election) also need to be garbage collected. The
latter seems to be used for the new leader to determine the last tiered
segment.
    default Iterator<RemoteLogSegmentMetadata>
listRemoteLogSegments(TopicPartition topicPartition)
    Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
topicPartition, long leaderEpoch);

Right,.that is what we are currently doing. We will update the
javadocs and wiki with that. Earlier, we did not want to remove the
segments which are not matched with leader epochs from the ladder
partition as they may be used later by a replica which can become a
leader (unclean leader election) and refer those segments. But that
may leak these segments in remote storage until the topic lifetime. We
decided to cleanup the segments with the oldest incase of size based
retention also.

5102. RSM:
5102.1 For methods like fetchLogSegmentData(), it seems that they can
use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.

It will be useful to have metadata for RSM to fetch log segment. It
may create location/path using id with other metadata too.

5102.2 In fetchLogSegmentData(), should we use long instead of Long?

Wanted to keep endPosition as optional to read till the end of the
segment and avoid sentinels.

5102.3 Why only some of the methods have default implementation and others
Don't?

Actually,  RSM will not have any default implementations. Those 3
methods were made default earlier for tests etc. Updated the wiki.

5102.4. Could we define RemoteLogSegmentMetadataUpdate
and DeletePartitionUpdate?

Sure, they will be added.


5102.5 LogSegmentData: It seems that it's easier to pass
in leaderEpochIndex as a ByteBuffer or byte array than a file since it will
be generated in memory.

Right, this is in plan.

5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset and
startOffset. For example, deleteRecords() could move the startOffset to the
middle of a segment. If we copy the full segment to remote storage, the
baseOffset and the startOffset will be different.

Good point. startOffset is baseOffset by default, if not set explicitly.

5102.7 Could we define all the public methods for RemoteLogSegmentMetadata
and LogSegmentData?

Sure, updated the wiki.

5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
inclusive/exclusive?

It is inclusive, will update.

5103. configs:
5103.1 Could we define the default value of non-required configs (e.g the
size of new thread pools)?

Sure, that makes sense.

5103.2 It seems that local.log.retention.ms should default to retention.ms,
instead of remote.log.retention.minutes. Similarly, it seems
that local.log.retention.bytes should default to segment.bytes.

Right, we do not have  remote.log.retention as we discussed earlier.
Thanks for catching the typo.

5103.3 remote.log.manager.thread.pool.size: The description says "used in
scheduling tasks to copy segments, fetch remote log indexes and clean up
remote log segments". However, there is a separate
config remote.log.reader.threads for fetching remote data. It's weird to
fetch remote index and log in different thread pools since both are used
for serving fetch requests.

Right, remote.log.manager.thread.pool is mainly used for copy/cleanup
activities. Fetch path always goes through remote.log.reader.threads.

5103.4 remote.log.manager.task.interval.ms: Is that the amount of time to
back off when there is no work to do? If so, perhaps it can be renamed as
backoff.ms.

This is the delay interval for each iteration. It may be renamed to
remote.log.manager.task.delay.ms

5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
so, they need to be listed in this section.

remote.log.manager.task.interval.ms is the process internal, retry
interval is missing in the configs, which will be updated in the KIP.

5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
remote log segments to avoid multiple index fetches from the remote
storage." Is the RLM in memory or on disk? If on disk, where is it stored?
Do we need a configuration to bound the size?

It is stored on disk. They are stored in a directory
`remote-log-index-cache` under log dir. We plan to have a config for
that instead of default. We will have a configuration for that.

5105. The KIP uses local-log-start-offset and Earliest Local Offset in
different places. It would be useful to standardize the terminology.

Sure.

5106. The section on "In BuildingRemoteLogAux state". It listed two options
without saying which option is chosen.
We already mentioned in the KIP that we chose option-2.

5107. Follower to leader transition: It has step 2, but not step 1.
Step-1 is there but it is not explicitly highlighted. It is previous
table to step-2.

5108. If a consumer fetches from the remote data and the remote storage is
not available, what error code is used in the fetch response?

Good point. We have not yet defined the error for this case. We need
to define an error message and send the same in fetch response.

5109. "ListOffsets: For timestamps >= 0, it returns the first message
offset whose timestamp is >= to the given timestamp in the request. That
means it checks in remote log time indexes first, after which local log
time indexes are checked." Could you document which method in RLMM is used
for this?

Okay.

5110. Stopreplica: "it sets all the remote log segment metadata of that
partition with a delete marker and publishes them to RLMM." This seems
outdated given the new topic deletion logic.

Will update with KIP-516 related points.

5111. "RLM follower fetches the earliest offset for the earliest leader
epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
leaderEpoch) and updates that as the log start offset." Do we need that
since replication propagates logStartOffset already?

Good point. Right, existing replication protocol takes care of
updating the followers’s log start offset received from the leader.

5112. Is the default maxWaitMs of 500ms enough for fetching from remote
storage?

Remote reads may fail within the current default wait time, but
subsequent fetches would be able to serve as that data is stored in
the local cache. This cache is currently implemented in RSMs. But we
plan to pull this into the remote log messaging layer in future.

5113. "Committed offsets can be stored in a local file to avoid reading the
messages again when a broker is restarted." Could you describe the format
and the location of the file? Also, could the same message be processed by
RLMM again after broker restart? If so, how do we handle that?

Sure, we will update in the KIP.

5114. Message format
5114.1 There are two records named RemoteLogSegmentMetadataRecord with
apiKey 0 and 1.

Nice catch, that was a typo. Fixed in the wiki.

5114.2 RemoteLogSegmentMetadataRecord: Could we document whether endOffset
is inclusive/exclusive?
It is inclusive, will update.

5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a bit
more? Is that the epoch of the leader when it copies the segment to remote
storage? Also, how will this field be used?

Right, this is the leader epoch of the broker which copied this
segment. This is helpful in reason about which broker copied the
segment to remote storage.

5114.4 EventTimestamp: Could you explain this a bit more? Each record in
Kafka already has a timestamp field. Could we just use that?

This is the  timestamp at which  the respective event occurred. Added
this  to RemoteLogSegmentMetadata as RLMM can be  any other
implementation. We thought about that but it looked cleaner to use at
the message structure level instead of getting that from the consumer
record and using that to build the respective event.


5114.5 SegmentSizeInBytes: Could this just be int32?

Right, it looks like config allows only int value >= 14.

5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
for compaction. Perhaps it can be renamed to sth like
RemotePartitionRemover.

I am fine with RemotePartitionRemover or RemoteLogDeletionManager(we
have other manager classes like RLM, RLMM).

5116. "RLC receives the delete_partition_marked and processes it if it is
not yet processed earlier." How does it know whether
delete_partition_marked has been processed earlier?

This is to handle duplicate delete_partition_marked events. RLC
internally maintains a state for the delete_partition events and if it
already has an existing event then it ignores if it is already being
processed.

5117. Should we add a new MessageFormatter to read the tier metadata topic?

Right, this is in plan but did not mention it in the KIP. This will be
useful for debugging purposes too.

5118. "Maximum remote log reader thread pool task queue size. If the task
queue is full, broker will stop reading remote log segments." What do we
return to the fetch request in this case?

We return an error response for that partition.

5119. It would be useful to list all things not supported in the first
version in a Future work or Limitations section. For example, compacted
topic, JBOD, changing remote.log.storage.enable from true to false, etc.

We already have a non-goals section which is filled with some of these
details. Do we need another limitations section?

Thanks,
Satish.

On Wed, Nov 4, 2020 at 11:27 PM Jun Rao <ju...@confluent.io> wrote:
>
> Hi, Satish,
>
> Thanks for the updated KIP. A few more comments below.
>
> 605.2 "Build the local leader epoch cache by cutting the leader epoch
> sequence received from remote storage to [LSO, ELO]." I mentioned an issue
> earlier. Suppose the leader's local start offset is 100. The follower finds
> a remote segment covering offset range [80, 120). The producerState with
> this remote segment is up to offset 120. To trim the producerState to
> offset 100 requires more work since one needs to download the previous
> producerState up to offset 80 and then replay the messages from 80 to 100.
> It seems that it's simpler in this case for the follower just to take the
> remote segment as it is and start fetching from offset 120.
>
> 5016. Just to echo what Kowshik was saying. It seems that
> RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
> partition, not on the replicas for the __remote_log_segment_metadata
> partition. It's not clear how the leader of __remote_log_segment_metadata
> obtains the metadata for remote segments for deletion.
>
> 5100. KIP-516 has been accepted and is being implemented now. Could you
> update the KIP based on topicID?
>
> 5101. RLMM: It would be useful to clarify how the following two APIs are
> used. According to the wiki, the former is used for topic deletion and the
> latter is used for retention. It seems that retention should use the former
> since remote segments without a matching epoch in the leader (potentially
> due to unclean leader election) also need to be garbage collected. The
> latter seems to be used for the new leader to determine the last tiered
> segment.
>     default Iterator<RemoteLogSegmentMetadata>
> listRemoteLogSegments(TopicPartition topicPartition)
>     Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
> topicPartition, long leaderEpoch);
>
> 5102. RSM:
> 5102.1 For methods like fetchLogSegmentData(), it seems that they can
> use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
> 5102.2 In fetchLogSegmentData(), should we use long instead of Long?
> 5102.3 Why only some of the methods have default implementation and others
> don't?
> 5102.4. Could we define RemoteLogSegmentMetadataUpdate
> and DeletePartitionUpdate?
> 5102.5 LogSegmentData: It seems that it's easier to pass
> in leaderEpochIndex as a ByteBuffer or byte array than a file since it will
> be generated in memory.
> 5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset and
> startOffset. For example, deleteRecords() could move the startOffset to the
> middle of a segment. If we copy the full segment to remote storage, the
> baseOffset and the startOffset will be different.
> 5102.7 Could we define all the public methods for RemoteLogSegmentMetadata
> and LogSegmentData?
> 5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
> inclusive/exclusive?
>
> 5103. configs:
> 5103.1 Could we define the default value of non-required configs (e.g the
> size of new thread pools)?
> 5103.2 It seems that local.log.retention.ms should default to retention.ms,
> instead of remote.log.retention.minutes. Similarly, it seems
> that local.log.retention.bytes should default to segment.bytes.
> 5103.3 remote.log.manager.thread.pool.size: The description says "used in
> scheduling tasks to copy segments, fetch remote log indexes and clean up
> remote log segments". However, there is a separate
> config remote.log.reader.threads for fetching remote data. It's weird to
> fetch remote index and log in different thread pools since both are used
> for serving fetch requests.
> 5103.4 remote.log.manager.task.interval.ms: Is that the amount of time to
> back off when there is no work to do? If so, perhaps it can be renamed as
> backoff.ms.
> 5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
> so, they need to be listed in this section.
>
> 5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
> remote log segments to avoid multiple index fetches from the remote
> storage." Is the RLM in memory or on disk? If on disk, where is it stored?
> Do we need a configuration to bound the size?
>
> 5105. The KIP uses local-log-start-offset and Earliest Local Offset in
> different places. It would be useful to standardize the terminology.
>
> 5106. The section on "In BuildingRemoteLogAux state". It listed two options
> without saying which option is chosen.
>
> 5107. Follower to leader transition: It has step 2, but not step 1.
>
> 5108. If a consumer fetches from the remote data and the remote storage is
> not available, what error code is used in the fetch response?
>
> 5109. "ListOffsets: For timestamps >= 0, it returns the first message
> offset whose timestamp is >= to the given timestamp in the request. That
> means it checks in remote log time indexes first, after which local log
> time indexes are checked." Could you document which method in RLMM is used
> for this?
>
> 5110. Stopreplica: "it sets all the remote log segment metadata of that
> partition with a delete marker and publishes them to RLMM." This seems
> outdated given the new topic deletion logic.
>
> 5111. "RLM follower fetches the earliest offset for the earliest leader
> epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
> leaderEpoch) and updates that as the log start offset." Do we need that
> since replication propagates logStartOffset already?
>
> 5112. Is the default maxWaitMs of 500ms enough for fetching from remote
> storage?
>
> 5113. "Committed offsets can be stored in a local file to avoid reading the
> messages again when a broker is restarted." Could you describe the format
> and the location of the file? Also, could the same message be processed by
> RLMM again after broker restart? If so, how do we handle that?
>
> 5114. Message format
> 5114.1 There are two records named RemoteLogSegmentMetadataRecord with
> apiKey 0 and 1.
> 5114.2 RemoteLogSegmentMetadataRecord: Could we document whether endOffset
> is inclusive/exclusive?
> 5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a bit
> more? Is that the epoch of the leader when it copies the segment to remote
> storage? Also, how will this field be used?
> 5114.4 EventTimestamp: Could you explain this a bit more? Each record in
> Kafka already has a timestamp field. Could we just use that?
> 5114.5 SegmentSizeInBytes: Could this just be int32?
>
> 5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
> for compaction. Perhaps it can be renamed to sth like
> RemotePartitionRemover.
>
> 5116. "RLC receives the delete_partition_marked and processes it if it is
> not yet processed earlier." How does it know whether
> delete_partition_marked has been processed earlier?
>
> 5117. Should we add a new MessageFormatter to read the tier metadata topic?
>
> 5118. "Maximum remote log reader thread pool task queue size. If the task
> queue is full, broker will stop reading remote log segments." What do we
> return to the fetch request in this case?
>
> 5119. It would be useful to list all things not supported in the first
> version in a Future work or Limitations section. For example, compacted
> topic, JBOD, changing remote.log.storage.enable from true to false, etc.
>
> Thanks,
>
> Jun
>
> On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <kp...@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 <sa...@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.duggana@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 <dh...@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 <ha...@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.duggana@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 (
> > > > > > > jun@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.duggana@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.ch@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
> > (
> > > > > > >>>> jun@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 (
> > > > > > >>>>> kafka@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 (
> > > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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
> > > > > > >>>>>>>
> > >

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

Posted by Jun Rao <ju...@confluent.io>.
Hi, Satish,

Thanks for the updated KIP. A few more comments below.

605.2 "Build the local leader epoch cache by cutting the leader epoch
sequence received from remote storage to [LSO, ELO]." I mentioned an issue
earlier. Suppose the leader's local start offset is 100. The follower finds
a remote segment covering offset range [80, 120). The producerState with
this remote segment is up to offset 120. To trim the producerState to
offset 100 requires more work since one needs to download the previous
producerState up to offset 80 and then replay the messages from 80 to 100.
It seems that it's simpler in this case for the follower just to take the
remote segment as it is and start fetching from offset 120.

5016. Just to echo what Kowshik was saying. It seems that
RLMM.onPartitionLeadershipChanges() is only called on the replicas for a
partition, not on the replicas for the __remote_log_segment_metadata
partition. It's not clear how the leader of __remote_log_segment_metadata
obtains the metadata for remote segments for deletion.

5100. KIP-516 has been accepted and is being implemented now. Could you
update the KIP based on topicID?

5101. RLMM: It would be useful to clarify how the following two APIs are
used. According to the wiki, the former is used for topic deletion and the
latter is used for retention. It seems that retention should use the former
since remote segments without a matching epoch in the leader (potentially
due to unclean leader election) also need to be garbage collected. The
latter seems to be used for the new leader to determine the last tiered
segment.
    default Iterator<RemoteLogSegmentMetadata>
listRemoteLogSegments(TopicPartition topicPartition)
    Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicPartition
topicPartition, long leaderEpoch);

5102. RSM:
5102.1 For methods like fetchLogSegmentData(), it seems that they can
use RemoteLogSegmentId instead of RemoteLogSegmentMetadata.
5102.2 In fetchLogSegmentData(), should we use long instead of Long?
5102.3 Why only some of the methods have default implementation and others
don't?
5102.4. Could we define RemoteLogSegmentMetadataUpdate
and DeletePartitionUpdate?
5102.5 LogSegmentData: It seems that it's easier to pass
in leaderEpochIndex as a ByteBuffer or byte array than a file since it will
be generated in memory.
5102.6 RemoteLogSegmentMetadata: It seems that it needs both baseOffset and
startOffset. For example, deleteRecords() could move the startOffset to the
middle of a segment. If we copy the full segment to remote storage, the
baseOffset and the startOffset will be different.
5102.7 Could we define all the public methods for RemoteLogSegmentMetadata
and LogSegmentData?
5102.8 Could we document whether endOffset in RemoteLogSegmentMetadata is
inclusive/exclusive?

5103. configs:
5103.1 Could we define the default value of non-required configs (e.g the
size of new thread pools)?
5103.2 It seems that local.log.retention.ms should default to retention.ms,
instead of remote.log.retention.minutes. Similarly, it seems
that local.log.retention.bytes should default to segment.bytes.
5103.3 remote.log.manager.thread.pool.size: The description says "used in
scheduling tasks to copy segments, fetch remote log indexes and clean up
remote log segments". However, there is a separate
config remote.log.reader.threads for fetching remote data. It's weird to
fetch remote index and log in different thread pools since both are used
for serving fetch requests.
5103.4 remote.log.manager.task.interval.ms: Is that the amount of time to
back off when there is no work to do? If so, perhaps it can be renamed as
backoff.ms.
5103.5 Are rlm_process_interval_ms and rlm_retry_interval_ms configs? If
so, they need to be listed in this section.

5104. "RLM maintains a bounded cache(possibly LRU) of the index files of
remote log segments to avoid multiple index fetches from the remote
storage." Is the RLM in memory or on disk? If on disk, where is it stored?
Do we need a configuration to bound the size?

5105. The KIP uses local-log-start-offset and Earliest Local Offset in
different places. It would be useful to standardize the terminology.

5106. The section on "In BuildingRemoteLogAux state". It listed two options
without saying which option is chosen.

5107. Follower to leader transition: It has step 2, but not step 1.

5108. If a consumer fetches from the remote data and the remote storage is
not available, what error code is used in the fetch response?

5109. "ListOffsets: For timestamps >= 0, it returns the first message
offset whose timestamp is >= to the given timestamp in the request. That
means it checks in remote log time indexes first, after which local log
time indexes are checked." Could you document which method in RLMM is used
for this?

5110. Stopreplica: "it sets all the remote log segment metadata of that
partition with a delete marker and publishes them to RLMM." This seems
outdated given the new topic deletion logic.

5111. "RLM follower fetches the earliest offset for the earliest leader
epoch by calling RLMM.earliestLogOffset(TopicPartition topicPartition, int
leaderEpoch) and updates that as the log start offset." Do we need that
since replication propagates logStartOffset already?

5112. Is the default maxWaitMs of 500ms enough for fetching from remote
storage?

5113. "Committed offsets can be stored in a local file to avoid reading the
messages again when a broker is restarted." Could you describe the format
and the location of the file? Also, could the same message be processed by
RLMM again after broker restart? If so, how do we handle that?

5114. Message format
5114.1 There are two records named RemoteLogSegmentMetadataRecord with
apiKey 0 and 1.
5114.2 RemoteLogSegmentMetadataRecord: Could we document whether endOffset
is inclusive/exclusive?
5114.3 RemoteLogSegmentMetadataRecord: Could you explain LeaderEpoch a bit
more? Is that the epoch of the leader when it copies the segment to remote
storage? Also, how will this field be used?
5114.4 EventTimestamp: Could you explain this a bit more? Each record in
Kafka already has a timestamp field. Could we just use that?
5114.5 SegmentSizeInBytes: Could this just be int32?

5115. RemoteLogCleaner(RLC): This could be confused with the log cleaner
for compaction. Perhaps it can be renamed to sth like
RemotePartitionRemover.

5116. "RLC receives the delete_partition_marked and processes it if it is
not yet processed earlier." How does it know whether
delete_partition_marked has been processed earlier?

5117. Should we add a new MessageFormatter to read the tier metadata topic?

5118. "Maximum remote log reader thread pool task queue size. If the task
queue is full, broker will stop reading remote log segments." What do we
return to the fetch request in this case?

5119. It would be useful to list all things not supported in the first
version in a Future work or Limitations section. For example, compacted
topic, JBOD, changing remote.log.storage.enable from true to false, etc.

Thanks,

Jun

On Tue, Oct 27, 2020 at 5:57 PM Kowshik Prakasam <kp...@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 <sa...@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.duggana@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 <dh...@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 <ha...@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.duggana@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 (
> > > > > > jun@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.duggana@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.ch@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
> (
> > > > > >>>> jun@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 (
> > > > > >>>>> kafka@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 (
> > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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 (
> > > > > >>>>>>> kprakasam@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.duggana@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.duggana@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.duggana@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 (
> > > > > >>>>>>> yingz@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 ( jun@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 (
> > > > > >>>>>>> yingz@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
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>>>>
> > > > > >>>>
> > > > > >>>>
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > > >>
> > > > > >
> > > > > >
> > > > > >
> >
>

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

Posted by Kowshik Prakasam <ko...@gmail.com>.
Hi Satish,

Thanks for your response.

5015. That makes sense, good point.

5019 and 5020. My 2 cents is that while you are implementing it, it will be
useful to update the KIP with details about the RocksDB-based design that
you envision. This will facilitate the discussions.


Cheers,
Kowshik



On Fri, Nov 6, 2020 at 5:45 AM Satish Duggana <sa...@gmail.com>
wrote:

> 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 <kp...@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.duggana@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.duggana@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 <dh...@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 <ha...@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.duggana@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 (
> > > > > > > jun@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.duggana@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.ch@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 (
> > > > > > >>>> jun@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 (
> > > > > > >>>>> kafka@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 (
> > > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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 (
> > > > > > >>>>>>> kprakasam@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.duggana@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.duggana@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.duggana@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
> (
> > > > > > >>>>>>> yingz@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 ( jun@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
> (
> > > > > > >>>>>>> yingz@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
> > > > > > >>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>>>>
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>
> > > > > > >>>
> > > > > > >>
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > > >
> > >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
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 <kp...@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 <sa...@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 <sa...@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 <dh...@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 <ha...@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.duggana@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 (
> > > > > > jun@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.duggana@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.ch@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 (
> > > > > >>>> jun@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 (
> > > > > >>>>> kafka@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 (
> > > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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 (
> > > > > >>>>>>> kprakasam@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.duggana@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.duggana@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.duggana@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 (
> > > > > >>>>>>> yingz@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 ( jun@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 (
> > > > > >>>>>>> yingz@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
> > > > > >>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>>>>
> > > > > >>>>
> > > > > >>>>
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > > >>
> > > > > >
> > > > > >
> > > > > >
> >

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

Posted by Kowshik Prakasam <kp...@confluent.io>.
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 <sa...@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 <sa...@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 <dh...@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 <ha...@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.duggana@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 (
> > > > > jun@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.duggana@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.ch@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 (
> > > > >>>> jun@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 (
> > > > >>>>> kafka@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 (
> > > > >>>>>> yingz@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.ch@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 ( kprakasam@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.ch@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 ( ch@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.duggana@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 (
> > > > >>>>>>> kprakasam@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.duggana@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.duggana@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.duggana@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 (
> > > > >>>>>>> yingz@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 ( jun@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 (
> > > > >>>>>>> yingz@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
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>>>
> > > > >>>
> > > > >>>
> > > > >>
> > > > >>
> > > > >
> > > > >
> > > > >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
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 <sa...@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 <dh...@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 <ha...@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.duggana@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 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 locally,
> > > > but msg 5 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 offsets are [0, 4] and 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 (
> > > > jun@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 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 locally,
> > > >> but msg 5 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.duggana@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.ch@gmail.com ) > wrote:
> > > >>>
> > > >>>
> > > >>>>
> > > >>>>
> > > >>>> Updated the KIP with Meeting Notes section
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/
> > > 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 (
> > > >>>> jun@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 (
> > > >>>>> kafka@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 (
> > > >>>>>> yingz@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.ch@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 ( kprakasam@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 ) > 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.ch@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 ( ch@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.duggana@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 ) >
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> > > >>>
> > > >>>
> > > >>>>
> > > >>>>>
> > > >>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> (
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> > > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > > 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 ) >
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > >>>
> > > >>>
> > > >>>
> > > >>>>
> > > >>>>>
> > > >>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> (
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> > > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > > 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 ) >
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> 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 (
> > > >>>>>>> kprakasam@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 ) >
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> 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.duggana@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.duggana@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 ) >
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > >>>
> > > >>>
> > > >>>
> > > >>>>
> > > >>>>>
> > > >>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> (
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> > > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > > 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.duggana@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 ) >
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > >>>
> > > >>>
> > > >>>
> > > >>>>
> > > >>>>>
> > > >>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> (
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> > > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>>
> > > >>>
> > > 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 (
> > > >>>>>>> yingz@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 ( jun@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 (
> > > >>>>>>> yingz@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
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>
> > > >>>
> > > >>
> > > >>
> > > >
> > > >
> > > >

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

Posted by Satish Duggana <sa...@gmail.com>.
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 <dh...@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 <ha...@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.duggana@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 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 locally,
> > > but msg 5 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 offsets are [0, 4] and 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 (
> > > jun@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 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 locally,
> > >> but msg 5 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.duggana@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.ch@gmail.com ) > wrote:
> > >>>
> > >>>
> > >>>>
> > >>>>
> > >>>> Updated the KIP with Meeting Notes section
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/
> > 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 (
> > >>>> jun@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 (
> > >>>>> kafka@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 (
> > >>>>>> yingz@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.ch@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 ( kprakasam@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 ) > 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.ch@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 ( ch@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.duggana@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 ) >
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> > >>>
> > >>>
> > >>>>
> > >>>>>
> > >>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> (
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>>
> > 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 ) >
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>>
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > >>>
> > >>>
> > >>>
> > >>>>
> > >>>>>
> > >>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> (
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>>
> > 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 ) >
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> 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 (
> > >>>>>>> kprakasam@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 ) >
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> 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.duggana@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.duggana@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 ) >
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>>
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > >>>
> > >>>
> > >>>
> > >>>>
> > >>>>>
> > >>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> (
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>>
> > 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.duggana@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 ) >
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>>
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > >>>
> > >>>
> > >>>
> > >>>>
> > >>>>>
> > >>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> (
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>>
> > 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 (
> > >>>>>>> yingz@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 ( jun@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 (
> > >>>>>>> yingz@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
> > >>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>
> > >>>
> > >>
> > >>
> > >
> > >
> > >


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

Posted by Dhruvil Shah <dh...@confluent.io>.
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 <ha...@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.duggana@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 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 locally,
> > but msg 5 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 offsets are [0, 4] and 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 (
> > jun@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 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 locally,
> >> but msg 5 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.duggana@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.ch@gmail.com ) > wrote:
> >>>
> >>>
> >>>>
> >>>>
> >>>> Updated the KIP with Meeting Notes section
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/
> 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 (
> >>>> jun@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 (
> >>>>> kafka@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 (
> >>>>>> yingz@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.ch@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 ( kprakasam@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 ) > 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.ch@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 ( ch@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.duggana@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 ) >
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> >>>
> >>>
> >>>>
> >>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> (
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>>
> 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 ) >
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >>>
> >>>
> >>>
> >>>>
> >>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> (
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>>
> 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 ) >
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> 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 (
> >>>>>>> kprakasam@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 ) >
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> 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.duggana@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.duggana@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 ) >
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >>>
> >>>
> >>>
> >>>>
> >>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> (
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>>
> 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.duggana@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 ) >
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >>>
> >>>
> >>>
> >>>>
> >>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> (
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
> >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>>
> >>>
> 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 (
> >>>>>>> yingz@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 ( jun@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 (
> >>>>>>> yingz@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
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>
> >>
> >
> >
> >

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

Posted by Harsha Ch <ha...@gmail.com>.
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.duggana@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 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 locally,
> but msg 5 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 offsets are [0, 4] and 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 (
> jun@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 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 locally,
>> but msg 5 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.duggana@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.ch@gmail.com ) > wrote:
>>> 
>>> 
>>>> 
>>>> 
>>>> Updated the KIP with Meeting Notes section
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ 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 (
>>>> jun@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 (
>>>>> kafka@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 (
>>>>>> yingz@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.ch@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 ( kprakasam@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 ) > 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.ch@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 ( ch@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.duggana@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 ) >
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> (
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> 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 ) >
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>>> 
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> (
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> 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 ) >
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 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 (
>>>>>>> kprakasam@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 ) >
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 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.duggana@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.duggana@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 ) >
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>>> 
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> (
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> 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.duggana@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 ) >
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>>> 
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> (
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ (
>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ )
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> 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 (
>>>>>>> yingz@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 ( jun@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 (
>>>>>>> yingz@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
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
> 
> 
>

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

Posted by Satish Duggana <sa...@gmail.com>.
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 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 locally,
but msg 5 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 offsets are [0, 4] and 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 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 <ju...@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 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 locally,
> but msg 5 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 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 <sa...@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 <ha...@gmail.com> wrote:
> > >
> > > Updated the KIP with Meeting Notes section
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
> > >
> > > On Tue, Aug 25, 2020 at 1:03 PM Jun Rao <ju...@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
> > > > ?
> > > >
> > > > Jun
> > > >
> > > > On Tue, Aug 25, 2020 at 11:12 AM Harsha Chintalapani <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
> > > > >
> > > > > 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 <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/
> > > > > > 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 <ha...@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>
> > > > > > 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> sync
> > meeting
> > > > > > tomorrow:
> > > > > >
> > > > > > 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 <ha...@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 > 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 ) > 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 ) > 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 ) >
> > > > > >
> > > > > > 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>
> > > > > >
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> > > > > >
> > > > > > (
> > > > > >
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > > > >
> > > > >
> > > >
> > 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>
> > > > > >
> > > > > >
> > > > >
> > > >
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > > >
> > > > > >
> > > > > > (
> > > > > >
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > > > >
> > > > >
> > > >
> > 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>
> > > > > >
> > > > > > 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/ ) ' 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 )
> > > > > >
> > > > > > 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>
> > > > > >
> > > > > > 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/ ) ' 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 )
> > > > > >
> > > > > > 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 ) > 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 ) >
> > > > > >
> > > > > > 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>
> > > > > >
> > > > > >
> > > > >
> > > >
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > > >
> > > > > >
> > > > > > (
> > > > > >
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > > > >
> > > > >
> > > >
> > 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 ) >
> > > > > >
> > > > > > 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>
> > > > > >
> > > > > >
> > > > >
> > > >
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > > >
> > > > > >
> > > > > > (
> > > > > >
> > > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > > > >
> > > > >
> > > >
> > 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 ) >
> > > > > >
> > > > > > 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 ) >
> > > > > >
> > > > > > 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 ) >
> > > > > >
> > > > > > 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
> > > > > >
> > > > > >
> > > > >
> > > >
> >
> >

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

Posted by Jun Rao <ju...@confluent.io>.
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 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 locally,
but msg 5 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 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 <sa...@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 <ha...@gmail.com> wrote:
> >
> > Updated the KIP with Meeting Notes section
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
> >
> > On Tue, Aug 25, 2020 at 1:03 PM Jun Rao <ju...@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
> > > ?
> > >
> > > Jun
> > >
> > > On Tue, Aug 25, 2020 at 11:12 AM Harsha Chintalapani <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
> > > >
> > > > 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 <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/
> > > > > 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 <ha...@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>
> > > > > 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> sync
> meeting
> > > > > tomorrow:
> > > > >
> > > > > 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 <ha...@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 > 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 ) > 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 ) > 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 ) >
> > > > >
> > > > > 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>
> > > > >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> > > > >
> > > > > (
> > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > > >
> > > >
> > >
> 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>
> > > > >
> > > > >
> > > >
> > >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > >
> > > > >
> > > > > (
> > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > > >
> > > >
> > >
> 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>
> > > > >
> > > > > 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/ ) ' 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 )
> > > > >
> > > > > 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>
> > > > >
> > > > > 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/ ) ' 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 )
> > > > >
> > > > > 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 ) > 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 ) >
> > > > >
> > > > > 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>
> > > > >
> > > > >
> > > >
> > >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > >
> > > > >
> > > > > (
> > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > > >
> > > >
> > >
> 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 ) >
> > > > >
> > > > > 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>
> > > > >
> > > > >
> > > >
> > >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > >
> > > > >
> > > > > (
> > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > > >
> > > >
> > >
> 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 ) >
> > > > >
> > > > > 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 ) >
> > > > >
> > > > > 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 ) >
> > > > >
> > > > > 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
> > > > >
> > > > >
> > > >
> > >
>
>

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

Posted by Satish Duggana <sa...@gmail.com>.
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 <ha...@gmail.com> wrote:
>
> Updated the KIP with Meeting Notes section
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
>
> On Tue, Aug 25, 2020 at 1:03 PM Jun Rao <ju...@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
> > ?
> >
> > Jun
> >
> > On Tue, Aug 25, 2020 at 11:12 AM Harsha Chintalapani <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
> > >
> > > 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 <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/
> > > > 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 <ha...@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>
> > > > 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> sync meeting
> > > > tomorrow:
> > > >
> > > > 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 <ha...@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 > 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 ) > 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 ) > 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 ) >
> > > >
> > > > 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>
> > > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> > > >
> > > > (
> > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > >
> > >
> > 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>
> > > >
> > > >
> > >
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > >
> > > >
> > > > (
> > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > >
> > >
> > 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>
> > > >
> > > > 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/ ) ' 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 )
> > > >
> > > > 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>
> > > >
> > > > 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/ ) ' 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 )
> > > >
> > > > 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 ) > 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 ) >
> > > >
> > > > 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>
> > > >
> > > >
> > >
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > >
> > > >
> > > > (
> > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > >
> > >
> > 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 ) >
> > > >
> > > > 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>
> > > >
> > > >
> > >
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > >
> > > >
> > > > (
> > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/
> > > >
> > >
> > 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 ) >
> > > >
> > > > 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 ) >
> > > >
> > > > 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 ) >
> > > >
> > > > 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
> > > >
> > > >
> > >
> >


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

Posted by Harsha Ch <ha...@gmail.com>.
Updated the KIP with Meeting Notes section
https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes

On Tue, Aug 25, 2020 at 1:03 PM Jun Rao <ju...@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
> ?
>
> Jun
>
> On Tue, Aug 25, 2020 at 11:12 AM Harsha Chintalapani <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
> >
> > 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 <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/
> > > 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 <ha...@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>
> > > 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> sync meeting
> > > tomorrow:
> > >
> > > 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 <ha...@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 > 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 ) > 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 ) > 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 ) >
> > >
> > > 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>
> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> > >
> > > (
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/
> > >
> >
> 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>
> > >
> > >
> >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > >
> > >
> > > (
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/
> > >
> >
> 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>
> > >
> > > 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/ ) ' 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 )
> > >
> > > 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>
> > >
> > > 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/ ) ' 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 )
> > >
> > > 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 ) > 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 ) >
> > >
> > > 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>
> > >
> > >
> >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > >
> > >
> > > (
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/
> > >
> >
> 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 ) >
> > >
> > > 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>
> > >
> > >
> >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > >
> > >
> > > (
> > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/
> > >
> >
> 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 ) >
> > >
> > > 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 ) >
> > >
> > > 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 ) >
> > >
> > > 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
> > >
> > >
> >
>

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

Posted by Jun Rao <ju...@confluent.io>.
Hi, Harsha,

Thanks for the summary. Could you add the summary and the recording link to
the last section of
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
?

Jun

On Tue, Aug 25, 2020 at 11:12 AM Harsha Chintalapani <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
>
> 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 <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/
> > 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 <ha...@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>
> > 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> sync meeting
> > tomorrow:
> >
> > 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 <ha...@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 > 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 ) > 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 ) > 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 ) >
> >
> > 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>
> > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> >
> > (
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/
> >
> 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>
> >
> >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >
> >
> > (
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/
> >
> 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>
> >
> > 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/ ) ' 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 )
> >
> > 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>
> >
> > 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/ ) ' 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 )
> >
> > 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 ) > 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 ) >
> >
> > 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>
> >
> >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >
> >
> > (
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/
> >
> 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 ) >
> >
> > 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>
> >
> >
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >
> >
> > (
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/
> >
> 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 ) >
> >
> > 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 ) >
> >
> > 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 ) >
> >
> > 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
> >
> >
>

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

Posted by Harsha Chintalapani <ka...@harsha.io>.
Thanks everyone for attending the meeting today.
Here is the recording
https://drive.google.com/file/d/14PRM7U0OopOOrJR197VlqvRX5SXNtmKj/view?usp=sharing

Notes:

   1. KIP is updated with follower fetch protocol and ready to reviewed
   2. Satish to capture schema of internal metadata topic in 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 <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/
> 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 <ha...@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 <kp...@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> sync meeting
> tomorrow:
>
> 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 <ha...@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 > 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 ) > 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 ) > 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 ) >
>
> 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>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
>
> (
>
> https://cwiki.apache.org/confluence/display/KAFKA/
> 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>
>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>
>
> (
>
> https://cwiki.apache.org/confluence/display/KAFKA/
> 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>
>
> 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/ ) ' 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 )
>
> 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>
>
> 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/ ) ' 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 )
>
> 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 ) > 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 ) >
>
> 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>
>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>
>
> (
>
> https://cwiki.apache.org/confluence/display/KAFKA/
> 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 ) >
>
> 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>
>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>
>
> (
>
> https://cwiki.apache.org/confluence/display/KAFKA/
> 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 ) >
>
> 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 ) >
>
> 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 ) >
>
> 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
>
>

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

Posted by Ying Zheng <yi...@uber.com.INVALID>.
We did some basic feature tests at Uber. The test cases and results are
shared in this google doc:
https://docs.google.com/spreadsheets/d/1XhNJqjzwXvMCcAOhEH0sSXU6RTvyoSf93DHF-YMfGLk/edit?usp=sharing

The performance test results were already shared in the KIP last month.

On Mon, Aug 24, 2020 at 11:10 AM Harsha Ch <ha...@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 <kp...@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> sync meeting
> > tomorrow:
> >
> 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 <ha...@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 > 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 ) > 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 ) > 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 ) >
> >> >>
> >> >>
> >> >>>
> >> >>>
> >> >>> 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>
> >> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> >> >> (
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/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>
> >>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >> >> (
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/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>
> >> >>
> >> >>
> >> >>>
> >> >>>>
> >> >>>>
> >> >>>> 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/ ) ' 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 )
> >> >>
> >> >>
> >> >>>
> >> >>>>
> >> >>>>
> >> >>>> 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>
> >> >>>
> >> >>>
> >> >>>>
> >> >>>>>
> >> >>>>>
> >> >>>>> 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/ ) ' 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 )
> >> >>>
> >> >>>
> >> >>>>
> >> >>>>>
> >> >>>>>
> >> >>>>> 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 ) > 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 ) >
> >> >>>>
> >> >>>>
> >> >>>>>
> >> >>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> 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>
> >>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >> >> (
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/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 ) >
> >> >>>>>>
> >> >>>>>>
> >> >>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> 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>
> >>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >> >> (
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/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 ) >
> >> >>>>
> >> >>>>
> >> >>>>>
> >> >>>>>>
> >> >>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> 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 ) >
> >> >>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>
> >> >>>>>
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>> 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 ) >
> >> >>>>>>
> >> >>>>>>
> >> >>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> 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
> >> >>>>>>>>>>>
> >
> >
>

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

Posted by Harsha Ch <ha...@gmail.com>.
"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 <kp...@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> sync meeting
> tomorrow:
> 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 <ha...@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 > 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 ) > 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 ) > 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 ) >
>> >>
>> >>
>> >>>
>> >>>
>> >>> 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>
>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
>> >> (
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/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>
>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>> >> (
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/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>
>> >>
>> >>
>> >>>
>> >>>>
>> >>>>
>> >>>> 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/ ) ' 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 )
>> >>
>> >>
>> >>>
>> >>>>
>> >>>>
>> >>>> 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>
>> >>>
>> >>>
>> >>>>
>> >>>>>
>> >>>>>
>> >>>>> 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/ ) ' 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 )
>> >>>
>> >>>
>> >>>>
>> >>>>>
>> >>>>>
>> >>>>> 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 ) > 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 ) >
>> >>>>
>> >>>>
>> >>>>>
>> >>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> 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>
>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>> >> (
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/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 ) >
>> >>>>>>
>> >>>>>>
>> >>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> 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>
>> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>> >> (
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/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 ) >
>> >>>>
>> >>>>
>> >>>>>
>> >>>>>>
>> >>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> 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 ) >
>> >>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>
>> >>>>>
>> >>>>
>> >>>>
>> >>>>
>> >>>> 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 ) >
>> >>>>>>
>> >>>>>>
>> >>>>>>>
>> >>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> 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
>> >>>>>>>>>>>
>
>

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

Posted by Kowshik Prakasam <kp...@confluent.io>.
Hi Harsha,

The following google doc contains a proposal for temporary agenda for the
KIP-405 <https://issues.apache.org/jira/browse/KIP-405> sync meeting
tomorrow:
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 <ha...@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 > 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 ) > 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 ) > 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 ) >
> >>
> >>
> >>>
> >>>
> >>> 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>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> >> (
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/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>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >> (
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/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>
> >>
> >>
> >>>
> >>>>
> >>>>
> >>>> 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/ ) ' 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 )
> >>
> >>
> >>>
> >>>>
> >>>>
> >>>> 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>
> >>>
> >>>
> >>>>
> >>>>>
> >>>>>
> >>>>> 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/ ) ' 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 )
> >>>
> >>>
> >>>>
> >>>>>
> >>>>>
> >>>>> 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 ) > 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 ) >
> >>>>
> >>>>
> >>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> 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>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >> (
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/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 ) >
> >>>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 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>
> %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >> (
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/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 ) >
> >>>>
> >>>>
> >>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 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 ) >
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>>
> >>>> 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 ) >
> >>>>>>
> >>>>>>
> >>>>>>>
> >>>>>>>>
> >>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 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
> >>>>>>>>>>>

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

Posted by Harsha Ch <ha...@gmail.com>.
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 > 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 ) > 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 ) > 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 ) >
>> 
>> 
>>> 
>>> 
>>> 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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
>> (
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
>> )
>> 
>> 
>> 
>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>> (
>> https://cwiki.apache.org/confluence/display/KAFKA/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
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> 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/ ) ' 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 )
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> 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
>>> 
>>> 
>>>> 
>>>>> 
>>>>> 
>>>>> 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/ ) ' 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 )
>>> 
>>> 
>>>> 
>>>>> 
>>>>> 
>>>>> 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 ) > 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 ) >
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>> (
>> https://cwiki.apache.org/confluence/display/KAFKA/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 ) >
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>> (
>> https://cwiki.apache.org/confluence/display/KAFKA/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 ) >
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 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 ) >
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> 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 ) >
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 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
>>>>>>>>>>>> unclean-leader-election.
>>>>>>>>>>>> 2) Since the user set local retention time (or local
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> retention
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> bytes), I
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> think we are expected to
>>>>>>>>>>>> keep that much local data when possible (avoid
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> truncating
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> all
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> local
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> data). But, as you said,
>>>>>>>>>>>> unclean leader elections are very rare, this may not
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> be a
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> big
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> problem.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> The current design uses the leader broker as
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> source-of-truth.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> This
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> is
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> consistent with the
>>>>>>>>>>>> existing Kafka behavior.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> By using remote storage as the source-of-truth, the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> follower
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> logic
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> can
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> be a
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> little simpler,
>>>>>>>>>>>> but the leader logic is going to be more complex.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> Overall,
>>> 
>>> 
>>>> 
>>>> 
>>>> I
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> don't
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> see
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> there many benefits
>>>>>>>>>>>> of using remote storage as the source-of-truth.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> jun@ confluent. io ( jun@confluent.io )
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> wrote:
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi, Satish,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks for the reply.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 1001. In your example, I was thinking that you could
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> just
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> download the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> latest leader epoch from the object store. After that
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> you
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> know
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> leader
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> should end with offset 1100. The leader will delete
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> all
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> its
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> local data
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> before offset 1000 and start accepting new messages
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> at
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> offset
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 1100.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Consumer requests for messages before offset 1100
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> will
>> 
>> 
>>> 
>>> 
>>> be
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> served
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> from
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> the
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> object store. The benefit with this approach is that
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> it's
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> simpler to
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> reason
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> about who is the source of truth. The downside is
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> slightly
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> increased
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> unavailability window during unclean leader election.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> Since
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> unclean
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> leader
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> elections are rare, I am not sure if this is a big
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> concern.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 1008. Yes, I think introducing sth like
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> local. retention. ms ( http://local.retention.ms/ )
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> seems more
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> consistent.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Jun
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> satish. duggana@ gmail. com ( satish.duggana@gmail.com )
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> HI Jun,
>>>>>>>>>>>>>> Thanks for your comments. We put our inline replies
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> below.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1001. I was thinking that you could just use the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> tiered
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> metadata to
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> do
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> the
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> reconciliation. The tiered metadata contains offset
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> ranges
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> and
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> epoch
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> history. Those should be enough for reconciliation
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> purposes.
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> If we use remote storage as the source-of-truth
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> during
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> unclean-leader-election, it's possible that after
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> reconciliation the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> remote storage will have more recent data than the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> new
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> leader's local
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> storage. For example, the new leader's latest
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> message
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> is
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> offset 1000,
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> while the remote storage has message 1100. In such
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> a
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> case,
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> new
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> leader will have to download the messages from 1001
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> to
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 1100,
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> before
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> accepting new messages from producers. Otherwise,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> there
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> would
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> be a
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> gap
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> in the local data between 1000 and 1101.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Moreover, with the current design, leader epoch
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> history is
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> stored in
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> remote storage, rather than the metadata topic. We
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> did
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> consider
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> saving
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> epoch history in remote segment metadata. But the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> concern
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> is
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> that
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> there is currently no limit for the epoch history
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> size.
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> Theoretically,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> if a user has a very long remote retention time and
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> there
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> are
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> very
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> frequent leadership changes, the leader epoch
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> history
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> can
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> become too
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> long to fit into a regular Kafka message.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1003.3 Having just a serverEndpoint string is
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> probably
>>> 
>>> 
>>>> 
>>>> 
>>>> not
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> enough.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Connecting to a Kafka cluster may need various
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> security
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> credentials.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> We
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> can
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> make RLMM configurable and pass in the properties
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> through
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> configure()
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> method. Ditto for RSM.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> RLMM and RSM are already configurable and they
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> take
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> properties which
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> start with "remote.log.metadata." and
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> " remote. log. storage ( http://remote.log.storage/ )."
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> respectively and a few others. We have
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> listener-name
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> as the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> config
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> for
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> RLMM and other properties(like security) can be
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> sent
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> as you
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> suggested.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> We will update the KIP with the details.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1008.1 We started with log.retention.hours and
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> log.retention.minutes,
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> and
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> added log. retention. ms ( http://log.retention.ms/ ) later. If we are
>>>>>>>>>>>>>> adding a
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> new
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> configuration,
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> ms
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> level config alone is enough and is simpler. We can
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> build
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> tools to
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> make
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> the
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> configuration at different granularities easier.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> The
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> definition of
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> log. retention. ms ( http://log.retention.ms/ ) is "The number of
>>>>>>>>>>>>>> milliseconds to
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> keep a
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> log
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> file
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> before
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> deleting it". The deletion is independent of
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> whether
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> tiering is
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> enabled
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> or
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> not. If this changes to just the local portion of
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> the
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> data, we
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> are
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> changing
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the meaning of an existing configuration.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> We are fine with either way. We can go with
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> log.retention.xxxx
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> as the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> effective log retention instead of local log
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> retention.
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> With
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> this
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> convention, we need to introduce
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> local.log.retention
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> instead
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> of
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> remote. log. retention. ms ( http://remote.log.retention.ms/ ) that we
>>>>>>>>>>>>>> proposed. If
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> log. retention. ms ( http://log.retention.ms/ )
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> as -1
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> then remote retention is also considered as
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> unlimited
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> but
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> user
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> should
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> be able to set the local. retention. ms ( http://local.retention.ms/ ).
>>>>>>>>>>>>>> So, we need to introduce local. log. retention. ms (
>>>>>>>>>>>>>> http://local.log.retention.ms/ )
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> and
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> local.log.retention.bytes which should always be
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> <=
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> log. retention. ms/ bytes ( http://log.retention.ms/bytes ) respectively.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> jun@ confluent. io ( jun@confluent.io ) >
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hi, Satish,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Thanks for the reply. A few quick comments below.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 1001. I was thinking that you could just use the
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> tiered
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> metadata to
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> do
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> reconciliation. The tiered metadata contains
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> offset
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> ranges
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> and
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> epoch
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> history. Those should be enough for
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> reconciliation
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> purposes.
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 1003.3 Having just a serverEndpoint string is
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> probably
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> not
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> enough.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Connecting to a Kafka cluster may need various
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> security
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> credentials.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> We
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> can
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> make RLMM configurable and pass in the properties
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> through the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> configure()
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> method. Ditto for RSM.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 1008.1 We started with log.retention.hours and
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> log.retention.minutes,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> and
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> added log. retention. ms ( http://log.retention.ms/ ) later. If we are
>>>>>>>>>>>>>>> adding a
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> new
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> configuration,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> ms
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> level config alone is enough and is simpler. We
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> can
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> build
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> tools to
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> make
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> configuration at different granularities easier.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> The
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> definition of
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> log. retention. ms ( http://log.retention.ms/ ) is "The number of
>>>>>>>>>>>>>>> milliseconds
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> to
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> keep
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> a
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> log file
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> before
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> deleting it". The deletion is independent of
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> whether
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> tiering
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> is
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> enabled
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> or
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> not. If this changes to just the local portion of
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> data,
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> we are
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> changing
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> the meaning of an existing configuration.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> satish. duggana@ gmail. com ( satish.duggana@gmail.com ) >
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Thank you for the comments! Ying, Harsha and I
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> discussed
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> and put
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> our
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> comments below.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1001. The KIP described a few scenarios of
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> unclean
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> leader
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> elections.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> This
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> is very useful, but I am wondering if this is
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> the
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> best
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> approach.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> My
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> understanding of the proposed approach is to
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> allow
>>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> new
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> (unclean)
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> to take new messages immediately. While this
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> increases
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> availability,
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> it
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> creates the problem that there could be
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> multiple
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> conflicting
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> segments
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the remote store for the same offset range.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> This
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> seems
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> to
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> make it
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> harder
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> for RLMM to determine which archived log
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> segments
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> contain
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> correct
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> data.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> For example, an archived log segment could at
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> one
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> time
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> be
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> correct
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> data,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> but be changed to incorrect data after an
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> unclean
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> leader
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> election.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> An
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> alternative approach is to let the unclean
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> leader
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> use
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> archived
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> data as
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the source of truth. So, when the new (unclean)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> leader
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> takes
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> over,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> it
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> first
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> reconciles the local data based on the archived
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> data
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> before
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> taking
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> new
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> messages. This makes the job of RLMM a bit
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> easier
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> since all
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> archived
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> data
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> are considered correct. This increases
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> availability a
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> bit.
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> However,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> since
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> unclean leader elections are rare, this may be
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> ok.
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Firstly, We don't want to assume the remote
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> storage is
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> more
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> reliable
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> than
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Kafka. Kafka unclean leader election usually
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> happens
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> when
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> there
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> is
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> a
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> large
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> scale outage that impacts multiple racks (or
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> even
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> multiple
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> availability
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> zones). In such a case, the remote storage may
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> be
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> unavailable or
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> unstable.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Pulling a large amount of data from the remote
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> storage
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> to
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> reconcile
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> the
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> local data may also exacerbate the outage. With
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> current
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> design,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the new
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> leader can start working even when the remote
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> storage
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> is
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> temporarily
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> unavailable.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Secondly, it is not easier to implement the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> reconciling
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> logic at
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> side. It can take a long time for the new
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> leader
>> 
>> 
>>> 
>>> 
>>> to
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> download the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> remote
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> data and rebuild local producer id / leader
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> epoch
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> information.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> During
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> period, the leader cannot accept any requests
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> from
>>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> clients
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> and
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> followers. We have to introduce a new state for
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> leader, and a
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> new
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> error
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> code to let the clients / followers know what
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> is
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> happening.
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1002. RemoteStorageManager.
>>>>>>>>>>>>>>>> 1002.1 There seems to be some inconsistencies
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> in
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> RemoteStorageManager.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> We
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> pass in RemoteLogSegmentId copyLogSegment().
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> For
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> all
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> other
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> methods,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> we
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> pass
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> in RemoteLogSegmentMetadata.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Nice catch, we can have the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> RemoteLogSegmentMetadata
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> for
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> copyLogSegment
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> too.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1002.2 Is endOffset in RemoteLogSegmentMetadata
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> inclusive
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> or
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> exclusive?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> It is inclusive.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1002.3 It seems that we need an api to get the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> leaderEpoch
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> history
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> for
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> partition.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Yes, updated the KIP with the new method.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1002.4 Could you define the type of
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> RemoteLogSegmentContext?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> This is removed in the latest code and it is
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> not
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> needed.
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1003 RemoteLogMetadataManager
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1003.1 I am not sure why we need both of the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> following
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> methods
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> in RemoteLogMetadataManager. Could we combine
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> them
>>> 
>>> 
>>>> 
>>>> 
>>>> into
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> one that
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> takes
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> offset and returns RemoteLogSegmentMetadata?
>>>>>>>>>>>>>>>> RemoteLogSegmentId
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> getRemoteLogSegmentId(TopicPartition
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> topicPartition,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> long offset) throws IOException;
>>>>>>>>>>>>>>>> RemoteLogSegmentMetadata
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> getRemoteLogSegmentMetadata(RemoteLogSegmentId
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> remoteLogSegmentId) throws IOException;
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Good point, these can be merged for now. I
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> guess
>> 
>> 
>>> 
>>> 
>>> we
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> needed
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> them
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> in
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> earlier
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> version of the implementation but it is not
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> needed
>>> 
>>> 
>>>> 
>>>> 
>>>> now.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1003.2 There seems to be some inconsistencies
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> in
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> methods
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> below. I
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> am
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> not sure why one takes RemoteLogSegmentMetadata
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> and the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> other
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> takes RemoteLogSegmentId.
>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> putRemoteLogSegmentData(RemoteLogSegmentMetadata
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> remoteLogSegmentMetadata) throws IOException;
>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> remoteLogSegmentId) throws IOException;
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> RLMM stores RemoteLogSegmentMetadata which is
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> identified by
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> RemoteLogsSegmentId. So, when it is added it
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> takes
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> RemoteLogSegmentMetadata. `delete` operation
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> needs
>>> 
>>> 
>>>> 
>>>> 
>>>> only
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> RemoteLogsSegmentId
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> as RemoteLogSegmentMetadata can be identified
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> with
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> RemoteLogsSegmentId.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1003.3 In void onServerStarted(final String
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> serverEndpoint), what
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> is serverEndpoint used for?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> This can be used by RLMM implementation to
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> connect
>>> 
>>> 
>>>> 
>>>> 
>>>> to
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> local
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> Kafka
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> cluster. Incase of default implementation, it
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> is
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> used
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> in
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> initializing
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> kafka clients connecting to the local cluster.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1004. It would be useful to document how all
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> the
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> new
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> APIs
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> are
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> being
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> For example, when is
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> RemoteLogSegmentMetadata.markedForDeletion
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> being
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> set
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> and used? How are
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> being
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> used?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> RLMM APIs are going through the changes and
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> they
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> should be
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> ready
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> in a
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> few
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> days. I will update the KIP and the mail
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> thread
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> once
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> they
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> are
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> ready.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1005. Handling partition deletion: The KIP says
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> "RLMM
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> will
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> eventually
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> delete these segments by using
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> RemoteStorageManager."
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Which
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> replica
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> does
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> this logic?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> This is a good point. When a topic is deleted,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> it
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> will
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> not
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> have
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> any
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> leader/followers to do the cleanup. We will
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> have
>> 
>> 
>>> 
>>> 
>>> a
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> cleaner
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> agent
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> on a
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> single broker in the cluster to do this
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> cleanup,
>> 
>> 
>>> 
>>> 
>>> we
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> plan
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> to add
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> that
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> in
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> controller broker.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1006. "If there are any failures in removing
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> remote log
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> segments
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> then
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> those
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> are stored in a specific topic (default as
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> __remote_segments_to_be_deleted)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> and user can consume the events(which contain
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> remote-log-segment-id)
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> from
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> that topic and clean them up from remote
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> storage.
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> "
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Not
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> sure if
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> it's
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> worth
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the complexity of adding another topic. Could
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> we
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> just
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> retry?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Sure, we can keep this simpler for now by
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> logging
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> an
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> error
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> after
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> retries.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> We can give users a better way to process this
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> in
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> future.
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Oneway
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> can
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> be a
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> dead letter topic which can be configured by
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> the
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> user.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1007. RemoteFetchPurgatory: Could we just reuse
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> existing
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> fetchPurgatory?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> We have 2 types of delayed operations waiting
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> for 2
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> different
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> events.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> DelayedFetch waits for new messages from
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> producers.
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> DelayedRemoteFetch
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> waits for the remote-storage-read-task to
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> finish.
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> When
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> either of
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 2
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> events happens, we only want to notify one type
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> of
>>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> delayed
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> operations.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> It would be inefficient to put 2 types of
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> delayed
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> operations in
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> one
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> purgatory, as the tryComplete() methods of the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> delayed
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> operations
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> can
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> triggered by irrelevant events.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1008. Configurations:
>>>>>>>>>>>>>>>> 1008.1 remote. log. retention. ms ( http://remote.log.retention.ms/ ) ,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> remote.log.retention.minutes,
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> remote.log.retention.hours: It seems that we
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> just
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> need
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> ms
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> one.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Also,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> are we changing the meaning of existing config
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> log. retention. ms ( http://log.retention.ms/ )
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> to
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> mean
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> local retention? For backward compatibility,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> it's
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> better
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> to not
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> change
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> meaning of existing configurations.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> We agree that we only need
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> remote. log. retention. ms ( http://remote.log.retention.ms/ )
>>> 
>>> 
>>>> 
>>>> 
>>>> .
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> But,
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> existing
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>> has 3 properties ( log. retention. ms ( http://log.retention.ms/ ) ,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> log.retention.minutes,
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> log.retention.hours). We just
>>>>>>>>>>>>>>>> want to keep consistent with the existing
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> properties.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Existing log.retention.xxxx config is about log
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> retention
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> in
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> broker’s
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> storage which is local. It should be easy for
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> users to
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> configure
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> storage with local retention and remote
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> retention
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> based on
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> their
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> usage.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1008.2 Should remote. log. storage. enable (
>>>>>>>>>>>>>>>> http://remote.log.storage.enable/ ) be at
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> the
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> topic
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> level?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> We can introduce topic level config for the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> same
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> remote.log
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> settings.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> User
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> can set the desired config while creating the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> topic.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> remote. log. storage. enable ( http://remote.log.storage.enable/ ) property
>>>>>>>>>>>>>>>> is not
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> allowed
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> to be
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> updated
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> after
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> topic is created. Other remote.log.* properties
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> can be
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> modified.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> We
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> support flipping remote. log. storage. enable (
>>>>>>>>>>>>>>>> http://remote.log.storage.enable/ ) in
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> next
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> versions.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1009. It would be useful to list all
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> limitations
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> in a
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> separate
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> section:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> compacted topic, JBOD, etc. Also, is changing a
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> topic
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> from
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> delete
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> to
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> compact and vice versa allowed when tiering is
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> enabled?
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> +1 to have limitations in a separate section.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> We
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> will
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> update the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> KIP
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> with
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> that.
>>>>>>>>>>>>>>>> Topic created with effective value for
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> remote.log.enabled
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> as
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> true,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> can not
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> change its retention policy from delete to
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> compact.
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1010. Thanks for performance numbers. Are those
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> with
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> RocksDB as
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> the
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> cache?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> No, We have not yet added RocksDB support. This
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> is
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> based on
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> in-memory
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> map
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> representation. We will add that support and
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> update
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> this
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> thread
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> after
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> updating the KIP with the numbers.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> Satish.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> jun@ confluent. io ( jun@confluent.io ) >
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Hi, Satish, Ying, Harsha,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Thanks for the updated KIP. A few more
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> comments
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> below.
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1000. Regarding Colin's question on querying
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> metadata
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> directly
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> in the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> remote block store. One issue is that not all
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> block
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> stores
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> offer
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> the
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> needed
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> api to query the metadata. For example, S3
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> only
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> offers
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> an api
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> to
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> list
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> objects under a prefix and this api has the
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> eventual
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> consistency
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> semantic.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1001. The KIP described a few scenarios of
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> unclean
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> leader
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> elections.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> This
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> is very useful, but I am wondering if this is
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> best
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> approach.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> My
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> understanding of the proposed approach is to
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> allow
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> new
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> (unclean)
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> to take new messages immediately. While this
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> increases
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> availability,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> creates the problem that there could be
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> multiple
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> conflicting
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> segments in
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> the remote store for the same offset range.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> This
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> seems
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> to make
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> it
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> harder
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> for RLMM to determine which archived log
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> segments
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> contain the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> correct
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> data.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> For example, an archived log segment could at
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> one
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> time
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> be the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> correct
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> data,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> but be changed to incorrect data after an
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> unclean
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> leader
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> election.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> An
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> alternative approach is to let the unclean
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> leader
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> use the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> archived
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> data
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> the source of truth. So, when the new
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> (unclean)
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> leader
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> takes
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> over,
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> it
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> reconciles the local data based on the
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> archived
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> data
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> before
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> taking
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> new
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> messages. This makes the job of RLMM a bit
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> easier
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> since
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> all
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> archived
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> data
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> are considered correct. This increases
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> availability a
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> bit.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> However,
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> since
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> unclean leader elections are rare, this may
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> be
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> ok.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1002. RemoteStorageManager.
>>>>>>>>>>>>>>>>> 1002.1 There seems to be some inconsistencies
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> in
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> RemoteStorageManager. We
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> pass in RemoteLogSegmentId copyLogSegment().
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> For
>>> 
>>> 
>>>> 
>>>> 
>>>> all
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> other
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> methods,
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> pass
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> in RemoteLogSegmentMetadata.
>>>>>>>>>>>>>>>>> 1002.2 Is endOffset in
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> RemoteLogSegmentMetadata
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> inclusive or
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> exclusive?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1002.3 It seems that we need an api to get
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> the
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> leaderEpoch
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> history
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> for a
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> partition.
>>>>>>>>>>>>>>>>> 1002.4 Could you define the type of
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> RemoteLogSegmentContext?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1003 RemoteLogMetadataManager
>>>>>>>>>>>>>>>>> 1003.1 I am not sure why we need both of the
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> following
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> methods
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> in RemoteLogMetadataManager. Could we combine
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> them
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> into
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> one
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> that
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> takes in
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> offset and returns RemoteLogSegmentMetadata?
>>>>>>>>>>>>>>>>> RemoteLogSegmentId
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> getRemoteLogSegmentId(TopicPartition
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> topicPartition,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> long offset) throws IOException;
>>>>>>>>>>>>>>>>> RemoteLogSegmentMetadata
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> getRemoteLogSegmentMetadata(RemoteLogSegmentId
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> remoteLogSegmentId) throws IOException;
>>>>>>>>>>>>>>>>> 1003.2 There seems to be some inconsistencies
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> in
>>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> methods
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> below.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I am
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> not sure why one takes
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> RemoteLogSegmentMetadata
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> and
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> other
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> takes RemoteLogSegmentId.
>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> putRemoteLogSegmentData(RemoteLogSegmentMetadata
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> remoteLogSegmentMetadata) throws IOException;
>>>>>>>>>>>>>>>>> void
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> remoteLogSegmentId) throws IOException;
>>>>>>>>>>>>>>>>> 1003.3 In void onServerStarted(final String
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> serverEndpoint),
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> what
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> is serverEndpoint used for?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1004. It would be useful to document how all
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>> 
>>>> new
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> APIs are
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> being
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> used.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> For example, when is
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> RemoteLogSegmentMetadata.markedForDeletion
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> being set
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> and used? How are
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> being
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> used?
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1005. Handling partition deletion: The KIP
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> says
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> "RLMM
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> will
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> eventually
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> delete these segments by using
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> RemoteStorageManager."
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Which
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> replica
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> does
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> this logic?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1006. "If there are any failures in removing
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> remote
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> log
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> segments
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> then
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> are stored in a specific topic (default as
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> __remote_segments_to_be_deleted)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> and user can consume the events(which contain
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> remote-log-segment-id)
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> from
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> that topic and clean them up from remote
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> storage. "
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Not
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> sure
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> if
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> it's
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> worth
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> the complexity of adding another topic. Could
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> we
>>> 
>>> 
>>>> 
>>>> 
>>>> just
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> retry?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1007. RemoteFetchPurgatory: Could we just
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> reuse
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> existing
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> fetchPurgatory?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1008. Configurations:
>>>>>>>>>>>>>>>>> 1008.1 remote. log. retention. ms ( http://remote.log.retention.ms/ ) ,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> remote.log.retention.minutes,
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> remote.log.retention.hours: It seems that we
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> just
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> need
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> the ms
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> one.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Also,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> are we changing the meaning of existing
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> config
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> log. retention. ms ( http://log.retention.ms/ )
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> to
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> mean
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> local retention? For backward compatibility,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> it's
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> better
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> to not
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> change
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> meaning of existing configurations.
>>>>>>>>>>>>>>>>> 1008.2 Should remote. log. storage. enable (
>>>>>>>>>>>>>>>>> http://remote.log.storage.enable/ ) be at
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> topic
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> level?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1009. It would be useful to list all
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> limitations
>>> 
>>> 
>>>> 
>>>> 
>>>> in a
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> separate
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> section:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> compacted topic, JBOD, etc. Also, is
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> changing a
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> topic
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> from
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> delete
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> to
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> compact and vice versa allowed when tiering
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> is
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> enabled?
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 1010. Thanks for performance numbers. Are
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> those
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> with
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> RocksDB as
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> cache?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> harsha. ch@ gmail. com ( harsha.ch@gmail.com )
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Hi Colin,
>>>>>>>>>>>>>>>>>> Thats not what we said in
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> the
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> previous
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> email.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> RLMM
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> pluggable storage and by running numbers
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> even
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> 1PB
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> data
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> you do
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> not
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> need
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> than 10GB local storage.
>>>>>>>>>>>>>>>>>> If in future this becomes a blocker for any
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> users
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> we
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> can
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> revisit
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> but
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> does not warrant another implementation at
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> this
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> point
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> to push
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> data
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> remote storage.
>>>>>>>>>>>>>>>>>> We can ofcourse implement another RLMM that
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> is
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> optional for
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> users
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> configure to push to remote. But that
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> doesn't
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> need
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> to
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> be
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> addressed
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>> Harsha
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> On Wed, Jul 15, 2020 at 5:50 PM Colin
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> McCabe
>> 
>> 
>>> 
>>> 
>>> <
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> cmccabe@ apache. org ( cmccabe@apache.org )
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Hi Ying,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Thanks for the response.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> It sounds like you agree that storing the
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> metadata
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> in the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> remote
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> would be a better design overall. Given
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> that
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> that's
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> true,
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> is
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> there
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> reason to include the worse
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> implementation
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> based
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> on
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> RocksDB?
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Choosing a long-term metadata store is
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> not
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> something
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> that
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> we
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> lightly. It can take users years to
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> migrate
>>> 
>>> 
>>>> 
>>>> 
>>>> from
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> metadata
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> store
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> other. I also don't think it's realistic
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> or
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> desirable for
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> users
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> their own metadata stores. Even assuming
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> that
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> they
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> could
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> do
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> a
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> good
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> job
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> this, it would create huge fragmentation
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> in
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Kafka
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> ecosystem.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> best,
>>>>>>>>>>>>>>>>>>> Colin
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> On Tue, Jul 14, 2020, at 09:39, Ying
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> Zheng
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> wrote:
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>> Hi Colin,
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Satish and I are still discussing some
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> details
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> about how
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> to
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> transactions / producer ids. Satish is
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> going to
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> make some
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> minor
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> RLMM API and other parts. Other than
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> that,
>>> 
>>> 
>>>> 
>>>> 
>>>> we
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> have
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> finished
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> updating
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> I agree with Colin that the current
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> design
>>> 
>>> 
>>>> 
>>>> 
>>>> of
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> using
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> rocksDB
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> is
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> optimal. But this design is simple and
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> should
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> work
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> for
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> almost
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> all
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> existing Kafka users. RLMM is a plugin.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> Users
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> can
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> replace
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> rocksDB
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> their own RLMM implementation, if
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> needed.
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> So, I
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> think we
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> can
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> keep
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> rocksDB
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> for now. What do you think?
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>> Ying
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> On Tue, Jul 7, 2020 at 10:35 AM Jun
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> Rao <
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> jun@ confluent. io ( jun@confluent.io ) >
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Hi, Ying,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Thanks for the update. It's good to
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> see
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> progress on
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> this.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Please
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> let us
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> know when you are done updating the
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> KIP
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> wiki.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> On Tue, Jul 7, 2020 at 10:13 AM Ying
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> Zheng
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> < yingz@ uber. com. invalid ( yingz@uber.com.invalid )
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> Satish and I have added more design
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> details in
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> KIP,
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> including
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> how to
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> keep consistency between replicas
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> (especially
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> when
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> there
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> is
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> leadership
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> changes / log truncations) and new
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> metrics. We
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> also
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> made
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> some
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> minor
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> changes in the doc. We will finish
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> the
>>> 
>>> 
>>>> 
>>>> 
>>>> KIP
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> changes in
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> next
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> couple of
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> days. We will let you know when we
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> are
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> done.
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Most of
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> already updated to the wiki KIP.
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> You
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> can
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> take
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> a look.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> But
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> final version yet.
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> As for the implementation, the code
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> is
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> mostly
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> done
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> and
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> we
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> had
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> feature tests / system tests. I
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> have
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> added
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> performance
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> test
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> results
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> the KIP. However the recent design
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> changes
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> (e.g.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> leader
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> info
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> management / log truncation / some
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> of
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> new
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> metrics)
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> have not
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> implemented yet. It will take
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> about 2
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> weeks
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> for us to
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> implement
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> after you
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> review and agree with those design
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> changes.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> On Tue, Jul 7, 2020 at 9:23 AM Jun
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> Rao
>>> 
>>> 
>>>> 
>>>> 
>>>> <
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> jun@ confluent. io ( jun@confluent.io )
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Hi, Satish, Harsha,
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Any new updates on the KIP? This
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> feature
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> is
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> one of
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> most
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> important
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> most requested features in Apache
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> Kafka
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> right now.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> It
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> helpful
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> we can make sustained progress on
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> this.
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Could you
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> share
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> how
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> far
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> along
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> the design/implementation right
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> now?
>>> 
>>> 
>>>> 
>>>> 
>>>> Is
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> there
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> anything
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> people
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> can help to get it across the
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> line?
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> As for "transactional support"
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> and
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> "follower
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> requests/replication", no
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> further comments from me as long
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> as
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> producer
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> state
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> epoch
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> be restored properly from the
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> object
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> store
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> when
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> needed.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Jun
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 9, 2020 at 3:39 AM
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> Satish
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Duggana <
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> satish. duggana@ gmail. com ( satish.duggana@gmail.com ) >
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> We did not want to add many
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> implementation
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> details
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> in
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> But we
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> decided to add them in the KIP
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> as
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> appendix
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> or
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> sub-sections(including
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> follower fetch protocol) to
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> describe
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> flow
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> with
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> the
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> main
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> cases.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> That will answer most of the
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> queries. I
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> will
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> update
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> on
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> mail
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> thread when the respective
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> sections are
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> updated.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>> Satish.
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> On Sat, Jun 6, 2020 at 7:49 PM
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Alexandre
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Dupriez
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> < alexandre. dupriez@ gmail. com ( alexandre.dupriez@gmail.com ) >
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> wrote:
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> Hi Satish,
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> A couple of questions
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> specific
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> to the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> section
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> "Follower
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> Requests/Replication", pages
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> 16:17
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> in the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> design
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> document
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> [1].
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900. It is mentioned that
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> followers
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> fetch
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> auxiliary
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> states
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> from the
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> remote storage.
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900.a Does the consistency
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> model
>>> 
>>> 
>>>> 
>>>> 
>>>> of
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> external
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> impacts
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> reads
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> of leader epochs and other
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> auxiliary
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> data?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900.b What are the benefits
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> of
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> using
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> a
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> mechanism
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> to
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> store
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> access
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> the leader epochs which is
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> different
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> from other
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> metadata
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> associated
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> tiered segments? What are the
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> benefits of
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> retrieving
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> information
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> on-demand from the follower
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> rather
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> than
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> relying
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> on
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> propagation
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> via
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> topic __remote_log_metadata?
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> What
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> are the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> advantages
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> over
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> dedicated control structure
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> (e.g. a
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> new
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> record
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> type)
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> propagated via
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> this topic? Since in the
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> document,
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> different
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> control
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> paths
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> operating in the system, how
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> are
>>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> metadata
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> stored
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> __remote_log_metadata [which
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> also
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> include the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> epoch
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> which offloaded a segment]
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> and
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> remote
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> auxiliary
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> states,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> kept in
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> sync?
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900.c A follower can
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> encounter
>> 
>> 
>>> 
>>> 
>>> an
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> OFFSET_MOVED_TO_TIERED_STORAGE.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Is
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> this in response to a Fetch
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> or
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> OffsetForLeaderEpoch
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> request?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900.d What happens if, after
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> a
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> follower
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> encountered
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> an
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> OFFSET_MOVED_TO_TIERED_STORAGE
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> response,
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> its
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> attempts to
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> retrieve
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> leader epochs fail (for
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> instance,
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> because the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> remote
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> temporarily unavailable)?
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> Does
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> follower
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> fallbacks to
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> mode
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> where
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> it ignores tiered segments,
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> and
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> applies
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> truncation
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> using
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> locally
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> available information? What
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> happens
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> when
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> access
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> to
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> remote
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> is restored? How is the
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> replica
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> lineage
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> inferred
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> by
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> remote
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> leader
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> epochs reconciled with the
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> follower's
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> replica
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> lineage,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> evolved? Does the follower
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> remember
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> fetching
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> auxiliary
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> states
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> failed
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> in the past and attempt
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> reconciliation?
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Is
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> there
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> a
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> plan
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> offer
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> different strategies in this
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> scenario,
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> configurable
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> via
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> configuration?
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900.e Is the leader epoch
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> cache
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> offloaded with
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> every
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> segment?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Or
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> a new checkpoint is detected?
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> If
>>> 
>>> 
>>>> 
>>>> 
>>>> that
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> information
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> is
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> always
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> offloaded to avoid
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> duplicating
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> data,
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> how
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> does
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> the
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> remote
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> satisfy the request to
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> retrieve
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> it?
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900.f Since the leader epoch
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> cache
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> covers the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> entire
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> replica
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> lineage,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> what happens if, after a
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> leader
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> epoch
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> cache
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> file
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> is
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> offloaded
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> given segment, the local
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> epoch
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> cache
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> is
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> truncated
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> [not
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> necessarily
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> a range of offset included in
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> tiered
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> segments]?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> How
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> are
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> remote
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> local leader epoch caches
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> kept
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> consistent?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900.g Consumer can also use
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> leader
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> epochs (e.g.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> to
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> enable
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> fencing
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> protect against stale
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> leaders).
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> What
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> differences
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> between consumer and follower
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> fetches?
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> Especially,
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> would
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> consumers
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> also fetch leader epoch
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> information
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> from
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> the
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> remote
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> storage?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 900.h Assume a newly elected
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> leader
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> of a
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> topic-partition
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> detects
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> recent segments are available
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> in
>>> 
>>> 
>>>> 
>>>> 
>>>> the
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> external
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> storage,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> epochs
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> its local epoch. Does it
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> ignore
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> these
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> segments
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> and
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> their
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> associated
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> epoch-to-offset vectors? Or
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> try
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> to
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> reconstruct
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> its
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> local
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> replica
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> lineage based on the data
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> remotely
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> available?
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>> Alexandre
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> https:/ / docs. google. com/ document/ d/ 18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/
>> edit?usp=sharing (
>> https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
>> )
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> Le jeu. 4 juin 2020 à 19:55,
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> Satish
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> Duggana <
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> satish. duggana@ gmail. com ( satish.duggana@gmail.com ) >
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> a écrit :
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Jun,
>>>>>>>>>>>>>>>>>>>>>>>>>> Please let us know if you
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> have
>>> 
>>> 
>>>> 
>>>> 
>>>> any
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> comments
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> on
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> "transactional
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> support"
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> and "follower
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> requests/replication"
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> mentioned
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> in
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> wiki.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>> Satish.
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Jun 2, 2020 at 9:25
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> PM
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Satish
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> Duggana <
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> satish. duggana@ gmail. com ( satish.duggana@gmail.com ) >
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> wrote:
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks Jun for your
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> comments.
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 100. It would be useful
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> to
>> 
>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> provide
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> more
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> details
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> on
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> apis
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> are used. Otherwise, it's kind
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> of
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> hard
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> to
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> really
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> assess
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> apis are sufficient/redundant.
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> A
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> few
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> examples
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> below.
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> We will update the wiki
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> and
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> let
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> you
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> know.
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 100.1 deleteRecords
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> seems
>> 
>> 
>>> 
>>> 
>>> to
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> only
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> advance
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> the
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> logStartOffset
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> Log. How does that trigger the
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> deletion of
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> remote
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> log
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> segments?
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> RLMTask for leader
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> partition
>>> 
>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> periodically
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> checks
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> remote log segments
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> earlier
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> to
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> logStartOffset
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> and the
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> respective
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> remote log segment
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> metadata
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> and
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> data
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> are
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> deleted
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> by
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> using
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> RLMM
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> RSM.
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 100.2 stopReplica with
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> deletion
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> is
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> used
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> in 2
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> cases
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> (a)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> replica
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> reassignment; (b) topic
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
>> 
>> deletion.
>> 
>> 
>>> 
>>>> 
>>>> 
>>>> We
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> only
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> want to
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> delete
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> tiered
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> metadata in the second case.
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>>> 
>>> Also,
>>> 
>>> 
>>>> 
>>>> 
>>>> in
>>>> 
>>>> 
>>>>> 
>>>>>> 
>>>>>> 
>>>>>> the
>>>>>> 
>>>>>> 
>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> second
>>>>>>>> 
>>>>>>>> 
>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> case,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> who
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> 
>> 
>> 
> 
> 
>

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

Posted by Alexandre Dupriez <al...@gmail.com>.
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 <ha...@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 <ju...@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>
> > 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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/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
> > > 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' 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
> > >
> > > 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
> > > > 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' 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
> > > >
> > > > 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 <ju...@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>
> > > > > > 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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > > > >
> > > > > > > Satish.
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana <
> > > > > satish.duggana@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > > > > >
> > > > > > > >
> > > > > > > > Satish.
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng
> > > <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 <ju...@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
> > > > > <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
> > > > > > > > > > > unclean-leader-election.
> > > > > > > > > > > 2) Since the user set local retention time (or local
> > > retention
> > > > > > > bytes), I
> > > > > > > > > > > think we are expected to
> > > > > > > > > > > keep that much local data when possible (avoid
> truncating
> > > all
> > > > > the
> > > > > > > local
> > > > > > > > > > > data). But, as you said,
> > > > > > > > > > > unclean leader elections are very rare, this may not
> be a
> > > big
> > > > > > > problem.
> > > > > > > > > > >
> > > > > > > > > > > The current design uses the leader broker as
> > > source-of-truth.
> > > > > This
> > > > > > > is
> > > > > > > > > > > consistent with the
> > > > > > > > > > > existing Kafka behavior.
> > > > > > > > > > >
> > > > > > > > > > > By using remote storage as the source-of-truth, the
> > > follower
> > > > > logic
> > > > > > > can
> > > > > > > > > > be a
> > > > > > > > > > > little simpler,
> > > > > > > > > > > but the leader logic is going to be more complex.
> > Overall,
> > > I
> > > > > don't
> > > > > > > see
> > > > > > > > > > > there many benefits
> > > > > > > > > > > of using remote storage as the source-of-truth.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <
> > jun@confluent.io
> > > >
> > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the reply.
> > > > > > > > > > > >
> > > > > > > > > > > > 1001. In your example, I was thinking that you could
> > just
> > > > > > > download the
> > > > > > > > > > > > latest leader epoch from the object store. After that
> > you
> > > > > know
> > > > > > > the
> > > > > > > > > > leader
> > > > > > > > > > > > should end with offset 1100. The leader will delete
> all
> > > its
> > > > > > > local data
> > > > > > > > > > > > before offset 1000 and start accepting new messages
> at
> > > offset
> > > > > > > 1100.
> > > > > > > > > > > > Consumer requests for messages before offset 1100
> will
> > be
> > > > > served
> > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > > > object store. The benefit with this approach is that
> > it's
> > > > > > > simpler to
> > > > > > > > > > > reason
> > > > > > > > > > > > about who is the source of truth. The downside is
> > > slightly
> > > > > > > increased
> > > > > > > > > > > > unavailability window during unclean leader election.
> > > Since
> > > > > > > unclean
> > > > > > > > > > > leader
> > > > > > > > > > > > elections are rare, I am not sure if this is a big
> > > concern.
> > > > > > > > > > > >
> > > > > > > > > > > > 1008. Yes, I think introducing sth like
> > > local.retention.ms
> > > > > > > seems more
> > > > > > > > > > > > consistent.
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > > > > > > > > > satish.duggana@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > HI Jun,
> > > > > > > > > > > > > Thanks for your comments. We put our inline replies
> > > below.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1001. I was thinking that you could just use the
> > tiered
> > > > > > > metadata to
> > > > > > > > > > do
> > > > > > > > > > > > the
> > > > > > > > > > > > > reconciliation. The tiered metadata contains offset
> > > ranges
> > > > > and
> > > > > > > epoch
> > > > > > > > > > > > > history. Those should be enough for reconciliation
> > > > > purposes.
> > > > > > > > > > > > >
> > > > > > > > > > > > > If we use remote storage as the source-of-truth
> > during
> > > > > > > > > > > > > unclean-leader-election, it's possible that after
> > > > > > > reconciliation the
> > > > > > > > > > > > > remote storage will have more recent data than the
> > new
> > > > > > > leader's local
> > > > > > > > > > > > > storage. For example, the new leader's latest
> message
> > > is
> > > > > > > offset 1000,
> > > > > > > > > > > > > while the remote storage has message 1100. In such
> a
> > > case,
> > > > > the
> > > > > > > new
> > > > > > > > > > > > > leader will have to download the messages from 1001
> > to
> > > > > 1100,
> > > > > > > before
> > > > > > > > > > > > > accepting new messages from producers. Otherwise,
> > there
> > > > > would
> > > > > > > be a
> > > > > > > > > > gap
> > > > > > > > > > > > > in the local data between 1000 and 1101.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Moreover, with the current design, leader epoch
> > > history is
> > > > > > > stored in
> > > > > > > > > > > > > remote storage, rather than the metadata topic. We
> > did
> > > > > consider
> > > > > > > > > > saving
> > > > > > > > > > > > > epoch history in remote segment metadata. But the
> > > concern
> > > > > is
> > > > > > > that
> > > > > > > > > > > > > there is currently no limit for the epoch history
> > size.
> > > > > > > > > > Theoretically,
> > > > > > > > > > > > > if a user has a very long remote retention time and
> > > there
> > > > > are
> > > > > > > very
> > > > > > > > > > > > > frequent leadership changes, the leader epoch
> history
> > > can
> > > > > > > become too
> > > > > > > > > > > > > long to fit into a regular Kafka message.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1003.3 Having just a serverEndpoint string is
> > probably
> > > not
> > > > > > > enough.
> > > > > > > > > > > > > Connecting to a Kafka cluster may need various
> > security
> > > > > > > credentials.
> > > > > > > > > > We
> > > > > > > > > > > > can
> > > > > > > > > > > > > make RLMM configurable and pass in the properties
> > > through
> > > > > the
> > > > > > > > > > > configure()
> > > > > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > > > > >
> > > > > > > > > > > > > RLMM and  RSM are already configurable and they
> take
> > > > > > > properties which
> > > > > > > > > > > > > start with "remote.log.metadata." and
> > > "remote.log.storage."
> > > > > > > > > > > > > respectively and a few others. We have
> listener-name
> > > as the
> > > > > > > config
> > > > > > > > > > for
> > > > > > > > > > > > > RLMM and other properties(like security) can be
> sent
> > > as you
> > > > > > > > > > suggested.
> > > > > > > > > > > > > We will update the KIP with the details.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > > > log.retention.minutes,
> > > > > > > > > > > and
> > > > > > > > > > > > > added log.retention.ms later. If we are adding a
> new
> > > > > > > configuration,
> > > > > > > > > > ms
> > > > > > > > > > > > > level config alone is enough and is simpler. We can
> > > build
> > > > > > > tools to
> > > > > > > > > > make
> > > > > > > > > > > > the
> > > > > > > > > > > > > configuration at different granularities easier.
> The
> > > > > > > definition of
> > > > > > > > > > > > > log.retention.ms is "The number of milliseconds to
> > > keep a
> > > > > log
> > > > > > > file
> > > > > > > > > > > > before
> > > > > > > > > > > > > deleting it". The deletion is independent of
> whether
> > > > > tiering is
> > > > > > > > > > enabled
> > > > > > > > > > > > or
> > > > > > > > > > > > > not. If this changes to just the local portion of
> the
> > > > > data, we
> > > > > > > are
> > > > > > > > > > > > changing
> > > > > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > > > > >
> > > > > > > > > > > > > We are fine with either way. We can go with
> > > > > log.retention.xxxx
> > > > > > > as the
> > > > > > > > > > > > > effective log retention instead of local log
> > retention.
> > > > > With
> > > > > > > this
> > > > > > > > > > > > > convention, we need to introduce
> local.log.retention
> > > > > instead
> > > > > > > of
> > > > > > > > > > > > > remote.log.retention.ms that we proposed. If
> > > > > log.retention.ms
> > > > > > > as -1
> > > > > > > > > > > > > then remote retention is also considered as
> unlimited
> > > but
> > > > > user
> > > > > > > should
> > > > > > > > > > > > > be able to set the local.retention.ms.
> > > > > > > > > > > > > So, we need to introduce local.log.retention.ms
> and
> > > > > > > > > > > > > local.log.retention.bytes which should  always  be
> <=
> > > > > > > > > > > > > log.retention.ms/bytes respectively.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <
> > > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for the reply. A few quick comments below.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1001. I was thinking that you could just use the
> > > tiered
> > > > > > > metadata to
> > > > > > > > > > > do
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > reconciliation. The tiered metadata contains
> offset
> > > > > ranges
> > > > > > > and
> > > > > > > > > > epoch
> > > > > > > > > > > > > > history. Those should be enough for
> reconciliation
> > > > > purposes.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1003.3 Having just a serverEndpoint string is
> > > probably
> > > > > not
> > > > > > > enough.
> > > > > > > > > > > > > > Connecting to a Kafka cluster may need various
> > > security
> > > > > > > > > > credentials.
> > > > > > > > > > > We
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > make RLMM configurable and pass in the properties
> > > > > through the
> > > > > > > > > > > > configure()
> > > > > > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > > > > > > log.retention.minutes,
> > > > > > > > > > > > and
> > > > > > > > > > > > > > added log.retention.ms later. If we are adding a
> > new
> > > > > > > > > > configuration,
> > > > > > > > > > > ms
> > > > > > > > > > > > > > level config alone is enough and is simpler. We
> can
> > > build
> > > > > > > tools to
> > > > > > > > > > > make
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > configuration at different granularities easier.
> > The
> > > > > > > definition of
> > > > > > > > > > > > > > log.retention.ms is "The number of milliseconds
> to
> > > keep
> > > > > a
> > > > > > > log file
> > > > > > > > > > > > > before
> > > > > > > > > > > > > > deleting it". The deletion is independent of
> > whether
> > > > > tiering
> > > > > > > is
> > > > > > > > > > > enabled
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > not. If this changes to just the local portion of
> > the
> > > > > data,
> > > > > > > we are
> > > > > > > > > > > > > changing
> > > > > > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thank you for the comments! Ying, Harsha and I
> > > > > discussed
> > > > > > > and put
> > > > > > > > > > > our
> > > > > > > > > > > > > > > comments below.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1001. The KIP described a few scenarios of
> > unclean
> > > > > leader
> > > > > > > > > > > elections.
> > > > > > > > > > > > > This
> > > > > > > > > > > > > > > is very useful, but I am wondering if this is
> the
> > > best
> > > > > > > approach.
> > > > > > > > > > My
> > > > > > > > > > > > > > > understanding of the proposed approach is to
> > allow
> > > the
> > > > > new
> > > > > > > > > > > (unclean)
> > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > to take new messages immediately. While this
> > > increases
> > > > > > > > > > > availability,
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > creates the problem that there could be
> multiple
> > > > > > > conflicting
> > > > > > > > > > > segments
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the remote store for the same offset range.
> This
> > > seems
> > > > > to
> > > > > > > make it
> > > > > > > > > > > > > harder
> > > > > > > > > > > > > > > for RLMM to determine which archived log
> segments
> > > > > contain
> > > > > > > the
> > > > > > > > > > > correct
> > > > > > > > > > > > > data.
> > > > > > > > > > > > > > > For example, an archived log segment could at
> one
> > > time
> > > > > be
> > > > > > > the
> > > > > > > > > > > correct
> > > > > > > > > > > > > data,
> > > > > > > > > > > > > > > but be changed to incorrect data after an
> unclean
> > > > > leader
> > > > > > > > > > election.
> > > > > > > > > > > An
> > > > > > > > > > > > > > > alternative approach is to let the unclean
> leader
> > > use
> > > > > the
> > > > > > > > > > archived
> > > > > > > > > > > > > data as
> > > > > > > > > > > > > > > the source of truth. So, when the new (unclean)
> > > leader
> > > > > > > takes
> > > > > > > > > > over,
> > > > > > > > > > > it
> > > > > > > > > > > > > first
> > > > > > > > > > > > > > > reconciles the local data based on the archived
> > > data
> > > > > before
> > > > > > > > > > taking
> > > > > > > > > > > > new
> > > > > > > > > > > > > > > messages. This makes the job of RLMM a bit
> easier
> > > > > since all
> > > > > > > > > > > archived
> > > > > > > > > > > > > data
> > > > > > > > > > > > > > > are considered correct. This increases
> > > availability a
> > > > > bit.
> > > > > > > > > > However,
> > > > > > > > > > > > > since
> > > > > > > > > > > > > > > unclean leader elections are rare, this may be
> > ok.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Firstly, We don't want to assume the remote
> > > storage is
> > > > > more
> > > > > > > > > > > reliable
> > > > > > > > > > > > > than
> > > > > > > > > > > > > > > Kafka. Kafka unclean leader election usually
> > > happens
> > > > > when
> > > > > > > there
> > > > > > > > > > is
> > > > > > > > > > > a
> > > > > > > > > > > > > large
> > > > > > > > > > > > > > > scale outage that impacts multiple racks (or
> even
> > > > > multiple
> > > > > > > > > > > > availability
> > > > > > > > > > > > > > > zones). In such a case, the remote storage may
> be
> > > > > > > unavailable or
> > > > > > > > > > > > > unstable.
> > > > > > > > > > > > > > > Pulling a large amount of data from the remote
> > > storage
> > > > > to
> > > > > > > > > > reconcile
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > local data may also exacerbate the outage. With
> > the
> > > > > current
> > > > > > > > > > design,
> > > > > > > > > > > > > the new
> > > > > > > > > > > > > > > leader can start working even when the remote
> > > storage
> > > > > is
> > > > > > > > > > > temporarily
> > > > > > > > > > > > > > > unavailable.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Secondly, it is not easier to implement the
> > > reconciling
> > > > > > > logic at
> > > > > > > > > > > the
> > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > side. It can take a long time for the new
> leader
> > to
> > > > > > > download the
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > data and rebuild local producer id / leader
> epoch
> > > > > > > information.
> > > > > > > > > > > During
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > period, the leader cannot accept any requests
> > from
> > > the
> > > > > > > clients
> > > > > > > > > > and
> > > > > > > > > > > > > > > followers. We have to introduce a new state for
> > the
> > > > > > > leader, and a
> > > > > > > > > > > new
> > > > > > > > > > > > > error
> > > > > > > > > > > > > > > code to let the clients / followers know what
> is
> > > > > happening.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > > > > 1002.1 There seems to be some inconsistencies
> in
> > > > > > > > > > > > RemoteStorageManager.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment().
> For
> > > all
> > > > > other
> > > > > > > > > > methods,
> > > > > > > > > > > > we
> > > > > > > > > > > > > pass
> > > > > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Nice catch, we can have the
> > > RemoteLogSegmentMetadata
> > > > > for
> > > > > > > > > > > > copyLogSegment
> > > > > > > > > > > > > > > too.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > > > > inclusive
> > > > > > > or
> > > > > > > > > > > > exclusive?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It is inclusive.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > > > > leaderEpoch
> > > > > > > > > > history
> > > > > > > > > > > > for
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > partition.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Yes, updated the KIP with the new method.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1002.4 Could you define the type of
> > > > > > > RemoteLogSegmentContext?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > This is removed in the latest code and it is
> not
> > > > > needed.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1003.1 I am not sure why we need both of the
> > > following
> > > > > > > methods
> > > > > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine
> > them
> > > into
> > > > > > > one that
> > > > > > > > > > > > takes
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > > > > >     RemoteLogSegmentId
> > > > > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Good point, these can be merged for now. I
> guess
> > we
> > > > > needed
> > > > > > > them
> > > > > > > > > > in
> > > > > > > > > > > > > earlier
> > > > > > > > > > > > > > > version of the implementation but it is not
> > needed
> > > now.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1003.2 There seems to be some inconsistencies
> in
> > > the
> > > > > > > methods
> > > > > > > > > > > below. I
> > > > > > > > > > > > > am
> > > > > > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata
> > > and the
> > > > > > > other
> > > > > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > > > > >     void
> > > > > putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > > > > >     void
> > > > > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > RLMM stores RemoteLogSegmentMetadata which is
> > > > > identified by
> > > > > > > > > > > > > > > RemoteLogsSegmentId. So, when it is added it
> > takes
> > > > > > > > > > > > > > > RemoteLogSegmentMetadata. `delete` operation
> > needs
> > > only
> > > > > > > > > > > > > RemoteLogsSegmentId
> > > > > > > > > > > > > > > as RemoteLogSegmentMetadata can be identified
> > with
> > > > > > > > > > > > RemoteLogsSegmentId.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > > > > > serverEndpoint), what
> > > > > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > This can be used by RLMM implementation to
> > connect
> > > to
> > > > > the
> > > > > > > local
> > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > cluster. Incase of default  implementation, it
> is
> > > used
> > > > > in
> > > > > > > > > > > > initializing
> > > > > > > > > > > > > > > kafka clients connecting to the local cluster.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1004. It would be useful to document how all
> the
> > > new
> > > > > APIs
> > > > > > > are
> > > > > > > > > > being
> > > > > > > > > > > > > used.
> > > > > > > > > > > > > > > For example, when is
> > > > > > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > > > > being
> > > > > > > > > > > > > set
> > > > > > > > > > > > > > > and used? How are
> > > > > > > > > > > > > > >
> > > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> being
> > > > > > > > > > > > used?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > RLMM APIs are going through the changes and
> they
> > > > > should be
> > > > > > > ready
> > > > > > > > > > > in a
> > > > > > > > > > > > > few
> > > > > > > > > > > > > > > days. I will update the KIP and the mail
> thread
> > > once
> > > > > they
> > > > > > > are
> > > > > > > > > > > ready.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1005. Handling partition deletion: The KIP says
> > > "RLMM
> > > > > will
> > > > > > > > > > > eventually
> > > > > > > > > > > > > > > delete these segments by using
> > > RemoteStorageManager."
> > > > > Which
> > > > > > > > > > replica
> > > > > > > > > > > > > does
> > > > > > > > > > > > > > > this logic?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > This is a good point. When a topic is deleted,
> it
> > > will
> > > > > not
> > > > > > > have
> > > > > > > > > > any
> > > > > > > > > > > > > > > leader/followers to do the cleanup. We will
> have
> > a
> > > > > cleaner
> > > > > > > agent
> > > > > > > > > > > on a
> > > > > > > > > > > > > > > single broker in the cluster to do this
> cleanup,
> > we
> > > > > plan
> > > > > > > to add
> > > > > > > > > > > that
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > controller broker.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1006. "If there are any failures in removing
> > > remote log
> > > > > > > segments
> > > > > > > > > > > then
> > > > > > > > > > > > > those
> > > > > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > > > > remote-log-segment-id)
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > that topic and clean them up from remote
> storage.
> > > "
> > > > > Not
> > > > > > > sure if
> > > > > > > > > > > it's
> > > > > > > > > > > > > worth
> > > > > > > > > > > > > > > the complexity of adding another topic. Could
> we
> > > just
> > > > > > > retry?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Sure, we can keep this simpler for now by
> logging
> > > an
> > > > > error
> > > > > > > after
> > > > > > > > > > > > > retries.
> > > > > > > > > > > > > > > We can give users a better way to process this
> in
> > > > > future.
> > > > > > > Oneway
> > > > > > > > > > > can
> > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > dead letter topic which can be configured by
> the
> > > user.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse
> > the
> > > > > > > existing
> > > > > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We have 2 types of delayed operations waiting
> > for 2
> > > > > > > different
> > > > > > > > > > > events.
> > > > > > > > > > > > > > > DelayedFetch waits for new messages from
> > producers.
> > > > > > > > > > > > DelayedRemoteFetch
> > > > > > > > > > > > > > > waits for the remote-storage-read-task to
> finish.
> > > When
> > > > > > > either of
> > > > > > > > > > > the
> > > > > > > > > > > > 2
> > > > > > > > > > > > > > > events happens, we only want to notify one type
> > of
> > > the
> > > > > > > delayed
> > > > > > > > > > > > > operations.
> > > > > > > > > > > > > > > It would be inefficient to put 2 types of
> delayed
> > > > > > > operations in
> > > > > > > > > > one
> > > > > > > > > > > > > > > purgatory, as the tryComplete() methods of the
> > > delayed
> > > > > > > operations
> > > > > > > > > > > can
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > triggered by irrelevant events.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > > > > > remote.log.retention.minutes,
> > > > > > > > > > > > > > > remote.log.retention.hours: It seems that we
> just
> > > need
> > > > > the
> > > > > > > ms
> > > > > > > > > > one.
> > > > > > > > > > > > > Also,
> > > > > > > > > > > > > > > are we changing the meaning of existing config
> > > > > > > log.retention.ms
> > > > > > > > > > to
> > > > > > > > > > > > > mean
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > local retention? For backward compatibility,
> it's
> > > > > better
> > > > > > > to not
> > > > > > > > > > > > change
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We agree that we only need
> > remote.log.retention.ms
> > > .
> > > > > But,
> > > > > > > the
> > > > > > > > > > > > existing
> > > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > configuration
> > > > > > > > > > > > > > > has 3 properties (log.retention.ms,
> > > > > log.retention.minutes,
> > > > > > > > > > > > > > > log.retention.hours). We just
> > > > > > > > > > > > > > > want to keep consistent with the existing
> > > properties.
> > > > > > > > > > > > > > > Existing log.retention.xxxx config is about log
> > > > > retention
> > > > > > > in
> > > > > > > > > > > broker’s
> > > > > > > > > > > > > > > storage which is local. It should be easy for
> > > users to
> > > > > > > configure
> > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > storage with local retention and remote
> retention
> > > > > based on
> > > > > > > their
> > > > > > > > > > > > usage.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at
> the
> > > topic
> > > > > > > level?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We can introduce topic level config for the
> same
> > > > > remote.log
> > > > > > > > > > > settings.
> > > > > > > > > > > > > User
> > > > > > > > > > > > > > > can set the desired config while creating the
> > > topic.
> > > > > > > > > > > > > > > remote.log.storage.enable property is not
> allowed
> > > to be
> > > > > > > updated
> > > > > > > > > > > after
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > topic is created. Other remote.log.* properties
> > > can be
> > > > > > > modified.
> > > > > > > > > > We
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > support flipping remote.log.storage.enable in
> > next
> > > > > > > versions.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1009. It would be useful to list all
> limitations
> > > in a
> > > > > > > separate
> > > > > > > > > > > > section:
> > > > > > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a
> > > topic
> > > > > from
> > > > > > > delete
> > > > > > > > > > > to
> > > > > > > > > > > > > > > compact and vice versa allowed when tiering is
> > > enabled?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 to have limitations in a separate section.
> We
> > > will
> > > > > > > update the
> > > > > > > > > > > KIP
> > > > > > > > > > > > > with
> > > > > > > > > > > > > > > that.
> > > > > > > > > > > > > > > Topic  created with effective value for
> > > > > remote.log.enabled
> > > > > > > as
> > > > > > > > > > true,
> > > > > > > > > > > > > can not
> > > > > > > > > > > > > > > change its retention policy from delete to
> > compact.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1010. Thanks for performance numbers. Are those
> > > with
> > > > > > > RocksDB as
> > > > > > > > > > the
> > > > > > > > > > > > > cache?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > No, We have not yet added RocksDB support. This
> > is
> > > > > based on
> > > > > > > > > > > in-memory
> > > > > > > > > > > > > map
> > > > > > > > > > > > > > > representation. We will add that support and
> > update
> > > > > this
> > > > > > > thread
> > > > > > > > > > > after
> > > > > > > > > > > > > > > updating the KIP with the numbers.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <
> > > > > jun@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for the updated KIP. A few more
> comments
> > > > > below.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1000. Regarding Colin's question on querying
> > the
> > > > > metadata
> > > > > > > > > > > directly
> > > > > > > > > > > > > in the
> > > > > > > > > > > > > > > > remote block store. One issue is that not all
> > > block
> > > > > > > stores
> > > > > > > > > > offer
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > needed
> > > > > > > > > > > > > > > > api to query the metadata. For example, S3
> only
> > > > > offers
> > > > > > > an api
> > > > > > > > > > to
> > > > > > > > > > > > list
> > > > > > > > > > > > > > > > objects under a prefix and this api has the
> > > eventual
> > > > > > > > > > consistency
> > > > > > > > > > > > > > > semantic.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1001. The KIP described a few scenarios of
> > > unclean
> > > > > leader
> > > > > > > > > > > > elections.
> > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > is very useful, but I am wondering if this is
> > the
> > > > > best
> > > > > > > > > > approach.
> > > > > > > > > > > My
> > > > > > > > > > > > > > > > understanding of the proposed approach is to
> > > allow
> > > > > the
> > > > > > > new
> > > > > > > > > > > > (unclean)
> > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > to take new messages immediately. While this
> > > > > increases
> > > > > > > > > > > > availability,
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > creates the problem that there could be
> > multiple
> > > > > > > conflicting
> > > > > > > > > > > > > segments in
> > > > > > > > > > > > > > > > the remote store for the same offset range.
> > This
> > > > > seems
> > > > > > > to make
> > > > > > > > > > it
> > > > > > > > > > > > > harder
> > > > > > > > > > > > > > > > for RLMM to determine which archived log
> > segments
> > > > > > > contain the
> > > > > > > > > > > > correct
> > > > > > > > > > > > > > > data.
> > > > > > > > > > > > > > > > For example, an archived log segment could at
> > one
> > > > > time
> > > > > > > be the
> > > > > > > > > > > > correct
> > > > > > > > > > > > > > > data,
> > > > > > > > > > > > > > > > but be changed to incorrect data after an
> > unclean
> > > > > leader
> > > > > > > > > > > election.
> > > > > > > > > > > > An
> > > > > > > > > > > > > > > > alternative approach is to let the unclean
> > leader
> > > > > use the
> > > > > > > > > > > archived
> > > > > > > > > > > > > data
> > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > the source of truth. So, when the new
> (unclean)
> > > > > leader
> > > > > > > takes
> > > > > > > > > > > over,
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > reconciles the local data based on the
> archived
> > > data
> > > > > > > before
> > > > > > > > > > > taking
> > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > messages. This makes the job of RLMM a bit
> > easier
> > > > > since
> > > > > > > all
> > > > > > > > > > > > archived
> > > > > > > > > > > > > data
> > > > > > > > > > > > > > > > are considered correct. This increases
> > > availability a
> > > > > > > bit.
> > > > > > > > > > > However,
> > > > > > > > > > > > > since
> > > > > > > > > > > > > > > > unclean leader elections are rare, this may
> be
> > > ok.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > > > > > 1002.1 There seems to be some inconsistencies
> > in
> > > > > > > > > > > > > RemoteStorageManager. We
> > > > > > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment().
> > For
> > > all
> > > > > > > other
> > > > > > > > > > > methods,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > pass
> > > > > > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > > > > > > 1002.2 Is endOffset in
> RemoteLogSegmentMetadata
> > > > > > > inclusive or
> > > > > > > > > > > > > exclusive?
> > > > > > > > > > > > > > > > 1002.3 It seems that we need an api to get
> the
> > > > > > > leaderEpoch
> > > > > > > > > > > history
> > > > > > > > > > > > > for a
> > > > > > > > > > > > > > > > partition.
> > > > > > > > > > > > > > > > 1002.4 Could you define the type of
> > > > > > > RemoteLogSegmentContext?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > > > > > > 1003.1 I am not sure why we need both of the
> > > > > following
> > > > > > > methods
> > > > > > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine
> > > them
> > > > > into
> > > > > > > one
> > > > > > > > > > that
> > > > > > > > > > > > > takes in
> > > > > > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > > > > > >     RemoteLogSegmentId
> > > > > > > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > > > > 1003.2 There seems to be some inconsistencies
> > in
> > > the
> > > > > > > methods
> > > > > > > > > > > below.
> > > > > > > > > > > > > I am
> > > > > > > > > > > > > > > > not sure why one takes
> RemoteLogSegmentMetadata
> > > and
> > > > > the
> > > > > > > other
> > > > > > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > > > > > >     void
> > > > > putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > > > > > >     void
> > > > > > > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > > > > > serverEndpoint),
> > > > > > > > > > what
> > > > > > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1004. It would be useful to document how all
> > the
> > > new
> > > > > > > APIs are
> > > > > > > > > > > being
> > > > > > > > > > > > > used.
> > > > > > > > > > > > > > > > For example, when is
> > > > > > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > > > > > > being set
> > > > > > > > > > > > > > > > and used? How are
> > > > > > > > > > > > > > > >
> > > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > > > > > > > > > being
> > > > > > > > > > > > > used?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1005. Handling partition deletion: The KIP
> says
> > > "RLMM
> > > > > > > will
> > > > > > > > > > > > eventually
> > > > > > > > > > > > > > > > delete these segments by using
> > > RemoteStorageManager."
> > > > > > > Which
> > > > > > > > > > > replica
> > > > > > > > > > > > > does
> > > > > > > > > > > > > > > > this logic?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1006. "If there are any failures in removing
> > > remote
> > > > > log
> > > > > > > > > > segments
> > > > > > > > > > > > then
> > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > > > > > remote-log-segment-id)
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > that topic and clean them up from remote
> > > storage.  "
> > > > > Not
> > > > > > > sure
> > > > > > > > > > if
> > > > > > > > > > > > it's
> > > > > > > > > > > > > > > worth
> > > > > > > > > > > > > > > > the complexity of adding another topic. Could
> > we
> > > just
> > > > > > > retry?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just
> reuse
> > > the
> > > > > > > existing
> > > > > > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > > > > > remote.log.retention.minutes,
> > > > > > > > > > > > > > > > remote.log.retention.hours: It seems that we
> > just
> > > > > need
> > > > > > > the ms
> > > > > > > > > > > one.
> > > > > > > > > > > > > Also,
> > > > > > > > > > > > > > > > are we changing the meaning of existing
> config
> > > > > > > > > > log.retention.ms
> > > > > > > > > > > to
> > > > > > > > > > > > > mean
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > local retention? For backward compatibility,
> > it's
> > > > > better
> > > > > > > to not
> > > > > > > > > > > > > change
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at
> > the
> > > > > topic
> > > > > > > level?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1009. It would be useful to list all
> > limitations
> > > in a
> > > > > > > separate
> > > > > > > > > > > > > section:
> > > > > > > > > > > > > > > > compacted topic, JBOD, etc. Also, is
> changing a
> > > topic
> > > > > > > from
> > > > > > > > > > delete
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > compact and vice versa allowed when tiering
> is
> > > > > enabled?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1010. Thanks for performance numbers. Are
> those
> > > with
> > > > > > > RocksDB as
> > > > > > > > > > > the
> > > > > > > > > > > > > > > cache?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <
> > > > > > > harsha.ch@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > >                Thats not what we said in
> the
> > > > > previous
> > > > > > > email.
> > > > > > > > > > > RLMM
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > pluggable storage and by running numbers
> even
> > > 1PB
> > > > > data
> > > > > > > you do
> > > > > > > > > > > not
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > than 10GB local storage.
> > > > > > > > > > > > > > > > > If in future this becomes a blocker for any
> > > users
> > > > > we
> > > > > > > can
> > > > > > > > > > > revisit
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > does not warrant another implementation at
> > this
> > > > > point
> > > > > > > to push
> > > > > > > > > > > the
> > > > > > > > > > > > > data
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > > We can ofcourse implement another RLMM that
> > is
> > > > > > > optional for
> > > > > > > > > > > users
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > configure to push to remote. But that
> doesn't
> > > need
> > > > > to
> > > > > > > be
> > > > > > > > > > > > addressed
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Harsha
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin
> McCabe
> > <
> > > > > > > > > > > cmccabe@apache.org
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi Ying,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks for the response.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > It sounds like you agree that storing the
> > > > > metadata
> > > > > > > in the
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > would be a better design overall.  Given
> > that
> > > > > that's
> > > > > > > true,
> > > > > > > > > > is
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > reason to include the worse
> implementation
> > > based
> > > > > on
> > > > > > > > > > RocksDB?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Choosing a long-term metadata store is
> not
> > > > > something
> > > > > > > that
> > > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > > lightly.  It can take users years to
> > migrate
> > > from
> > > > > > > metadata
> > > > > > > > > > > > store
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > other.  I also don't think it's realistic
> > or
> > > > > > > desirable for
> > > > > > > > > > > > users
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > write
> > > > > > > > > > > > > > > > > > their own metadata stores.  Even assuming
> > > that
> > > > > they
> > > > > > > could
> > > > > > > > > > do
> > > > > > > > > > > a
> > > > > > > > > > > > > good
> > > > > > > > > > > > > > > job
> > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > this, it would create huge fragmentation
> in
> > > the
> > > > > Kafka
> > > > > > > > > > > > ecosystem.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying
> Zheng
> > > wrote:
> > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Satish and I are still discussing some
> > > details
> > > > > > > about how
> > > > > > > > > > to
> > > > > > > > > > > > > handle
> > > > > > > > > > > > > > > > > > > transactions / producer ids. Satish is
> > > going to
> > > > > > > make some
> > > > > > > > > > > > minor
> > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > RLMM API and other parts. Other than
> > that,
> > > we
> > > > > have
> > > > > > > > > > finished
> > > > > > > > > > > > > > > updating
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I agree with Colin that the current
> > design
> > > of
> > > > > using
> > > > > > > > > > rocksDB
> > > > > > > > > > > > is
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > optimal. But this design is simple and
> > > should
> > > > > work
> > > > > > > for
> > > > > > > > > > > almost
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > existing Kafka users. RLMM is a plugin.
> > > Users
> > > > > can
> > > > > > > replace
> > > > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > their own RLMM implementation, if
> needed.
> > > So, I
> > > > > > > think we
> > > > > > > > > > > can
> > > > > > > > > > > > > keep
> > > > > > > > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > > > > > > for now. What do you think?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Ying
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun
> Rao <
> > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks for the update. It's good to
> see
> > > the
> > > > > > > progress on
> > > > > > > > > > > > this.
> > > > > > > > > > > > > > > > Please
> > > > > > > > > > > > > > > > > > let us
> > > > > > > > > > > > > > > > > > > > know when you are done updating the
> KIP
> > > wiki.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying
> > > Zheng
> > > > > > > > > > > > > > > <yingz@uber.com.invalid
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Satish and I have added more design
> > > > > details in
> > > > > > > the
> > > > > > > > > > KIP,
> > > > > > > > > > > > > > > including
> > > > > > > > > > > > > > > > > > how to
> > > > > > > > > > > > > > > > > > > > > keep consistency between replicas
> > > > > (especially
> > > > > > > when
> > > > > > > > > > > there
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > leadership
> > > > > > > > > > > > > > > > > > > > > changes / log truncations) and new
> > > > > metrics. We
> > > > > > > also
> > > > > > > > > > > made
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > minor
> > > > > > > > > > > > > > > > > > > > > changes in the doc. We will finish
> > the
> > > KIP
> > > > > > > changes in
> > > > > > > > > > > the
> > > > > > > > > > > > > next
> > > > > > > > > > > > > > > > > > couple of
> > > > > > > > > > > > > > > > > > > > > days. We will let you know when we
> > are
> > > > > done.
> > > > > > > Most of
> > > > > > > > > > > the
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > already updated to the wiki KIP.
> You
> > > can
> > > > > take
> > > > > > > a look.
> > > > > > > > > > > But
> > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > As for the implementation, the code
> > is
> > > > > mostly
> > > > > > > done
> > > > > > > > > > and
> > > > > > > > > > > we
> > > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > had
> > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > feature tests / system tests. I
> have
> > > added
> > > > > the
> > > > > > > > > > > > performance
> > > > > > > > > > > > > test
> > > > > > > > > > > > > > > > > > results
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > the KIP. However the recent design
> > > changes
> > > > > > > (e.g.
> > > > > > > > > > leader
> > > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > management / log truncation / some
> of
> > > the
> > > > > new
> > > > > > > > > > metrics)
> > > > > > > > > > > > > have not
> > > > > > > > > > > > > > > > > been
> > > > > > > > > > > > > > > > > > > > > implemented yet. It will take
> about 2
> > > weeks
> > > > > > > for us to
> > > > > > > > > > > > > implement
> > > > > > > > > > > > > > > > > > after you
> > > > > > > > > > > > > > > > > > > > > review and agree with those design
> > > changes.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun
> > Rao
> > > <
> > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Any new updates on the KIP? This
> > > feature
> > > > > is
> > > > > > > one of
> > > > > > > > > > > the
> > > > > > > > > > > > > most
> > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > most requested features in Apache
> > > Kafka
> > > > > > > right now.
> > > > > > > > > > It
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > helpful
> > > > > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > > > we can make sustained progress on
> > > this.
> > > > > > > Could you
> > > > > > > > > > > share
> > > > > > > > > > > > > how
> > > > > > > > > > > > > > > far
> > > > > > > > > > > > > > > > > > along
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > the design/implementation right
> > now?
> > > Is
> > > > > there
> > > > > > > > > > > anything
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > > > > can help to get it across the
> line?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > As for "transactional support"
> and
> > > > > "follower
> > > > > > > > > > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > > > > > > > > > further comments from me as long
> as
> > > the
> > > > > > > producer
> > > > > > > > > > > state
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > be restored properly from the
> > object
> > > > > store
> > > > > > > when
> > > > > > > > > > > needed.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM
> > Satish
> > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > We did not want to add many
> > > > > implementation
> > > > > > > > > > details
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > But we
> > > > > > > > > > > > > > > > > > > > > > > decided to add them in the KIP
> as
> > > > > appendix
> > > > > > > or
> > > > > > > > > > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > > > > > > > > > follower fetch protocol) to
> > > describe
> > > > > the
> > > > > > > flow
> > > > > > > > > > with
> > > > > > > > > > > > the
> > > > > > > > > > > > > main
> > > > > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > > > > > That will answer most of the
> > > queries. I
> > > > > > > will
> > > > > > > > > > update
> > > > > > > > > > > > on
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > mail
> > > > > > > > > > > > > > > > > > > > > > > thread when the respective
> > > sections are
> > > > > > > updated.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM
> > > > > Alexandre
> > > > > > > Dupriez
> > > > > > > > > > > > > > > > > > > > > > > <al...@gmail.com>
> > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > A couple of questions
> specific
> > > to the
> > > > > > > section
> > > > > > > > > > > > > "Follower
> > > > > > > > > > > > > > > > > > > > > > > > Requests/Replication", pages
> > > 16:17
> > > > > in the
> > > > > > > > > > design
> > > > > > > > > > > > > document
> > > > > > > > > > > > > > > > > [1].
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900. It is mentioned that
> > > followers
> > > > > fetch
> > > > > > > > > > > auxiliary
> > > > > > > > > > > > > > > states
> > > > > > > > > > > > > > > > > > from the
> > > > > > > > > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900.a Does the consistency
> > model
> > > of
> > > > > the
> > > > > > > > > > external
> > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > impacts
> > > > > > > > > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > > > > > > > > > of leader epochs and other
> > > auxiliary
> > > > > > > data?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900.b What are the benefits
> of
> > > using
> > > > > a
> > > > > > > > > > mechanism
> > > > > > > > > > > to
> > > > > > > > > > > > > store
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > > > > > > > > the leader epochs which is
> > > different
> > > > > > > from other
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > tiered segments? What are the
> > > > > benefits of
> > > > > > > > > > > > retrieving
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > > > on-demand from the follower
> > > rather
> > > > > than
> > > > > > > relying
> > > > > > > > > > > on
> > > > > > > > > > > > > > > > > propagation
> > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > topic __remote_log_metadata?
> > What
> > > > > are the
> > > > > > > > > > > > advantages
> > > > > > > > > > > > > over
> > > > > > > > > > > > > > > > > > using a
> > > > > > > > > > > > > > > > > > > > > > > > dedicated control structure
> > > (e.g. a
> > > > > new
> > > > > > > record
> > > > > > > > > > > > type)
> > > > > > > > > > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > > > > > > > > > this topic? Since in the
> > > document,
> > > > > > > different
> > > > > > > > > > > > control
> > > > > > > > > > > > > > > paths
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > operating in the system, how
> > are
> > > the
> > > > > > > metadata
> > > > > > > > > > > > stored
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > __remote_log_metadata [which
> > also
> > > > > > > include the
> > > > > > > > > > > epoch
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > > > which offloaded a segment]
> and
> > > the
> > > > > remote
> > > > > > > > > > > auxiliary
> > > > > > > > > > > > > > > states,
> > > > > > > > > > > > > > > > > > kept in
> > > > > > > > > > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900.c A follower can
> encounter
> > an
> > > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > > > > > > > > > this in response to a Fetch
> or
> > > > > > > > > > > OffsetForLeaderEpoch
> > > > > > > > > > > > > > > > request?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900.d What happens if, after
> a
> > > > > follower
> > > > > > > > > > > encountered
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > > >
> OFFSET_MOVED_TO_TIERED_STORAGE
> > > > > response,
> > > > > > > its
> > > > > > > > > > > > > attempts to
> > > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > > > leader epochs fail (for
> > instance,
> > > > > > > because the
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > temporarily unavailable)?
> Does
> > > the
> > > > > > > follower
> > > > > > > > > > > > > fallbacks to
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > mode
> > > > > > > > > > > > > > > > > > > > where
> > > > > > > > > > > > > > > > > > > > > > > > it ignores tiered segments,
> and
> > > > > applies
> > > > > > > > > > > truncation
> > > > > > > > > > > > > using
> > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > > > > > > available information? What
> > > happens
> > > > > when
> > > > > > > access
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > > > > > is restored? How is the
> replica
> > > > > lineage
> > > > > > > > > > inferred
> > > > > > > > > > > by
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > > > epochs reconciled with the
> > > follower's
> > > > > > > replica
> > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > > > evolved? Does the follower
> > > remember
> > > > > > > fetching
> > > > > > > > > > > > > auxiliary
> > > > > > > > > > > > > > > > states
> > > > > > > > > > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > > > > > > > > > in the past and attempt
> > > > > reconciliation?
> > > > > > > Is
> > > > > > > > > > there
> > > > > > > > > > > a
> > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > offer
> > > > > > > > > > > > > > > > > > > > > > > > different strategies in this
> > > > > scenario,
> > > > > > > > > > > configurable
> > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900.e Is the leader epoch
> cache
> > > > > > > offloaded with
> > > > > > > > > > > > every
> > > > > > > > > > > > > > > > segment?
> > > > > > > > > > > > > > > > > > Or
> > > > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > > > > a new checkpoint is detected?
> > If
> > > that
> > > > > > > > > > information
> > > > > > > > > > > > is
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > always
> > > > > > > > > > > > > > > > > > > > > > > > offloaded to avoid
> duplicating
> > > data,
> > > > > how
> > > > > > > does
> > > > > > > > > > the
> > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > > > > > satisfy the request to
> retrieve
> > > it?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900.f Since the leader epoch
> > > cache
> > > > > > > covers the
> > > > > > > > > > > > entire
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > > > > > > > > > what happens if, after a
> leader
> > > epoch
> > > > > > > cache
> > > > > > > > > > file
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > offloaded
> > > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > given segment, the local
> epoch
> > > cache
> > > > > is
> > > > > > > > > > truncated
> > > > > > > > > > > > > [not
> > > > > > > > > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > a range of offset included in
> > > tiered
> > > > > > > segments]?
> > > > > > > > > > > How
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > local leader epoch caches
> kept
> > > > > > > consistent?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900.g Consumer can also use
> > > leader
> > > > > > > epochs (e.g.
> > > > > > > > > > > to
> > > > > > > > > > > > > enable
> > > > > > > > > > > > > > > > > > fencing
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > protect against stale
> leaders).
> > > What
> > > > > > > > > > differences
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > between consumer and follower
> > > > > fetches?
> > > > > > > > > > > Especially,
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > > > > > > > > also fetch leader epoch
> > > information
> > > > > from
> > > > > > > the
> > > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 900.h Assume a newly elected
> > > leader
> > > > > of a
> > > > > > > > > > > > > topic-partition
> > > > > > > > > > > > > > > > > > detects
> > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > > > recent segments are available
> > in
> > > the
> > > > > > > external
> > > > > > > > > > > > > storage,
> > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > epochs
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > its local epoch. Does it
> ignore
> > > these
> > > > > > > segments
> > > > > > > > > > > and
> > > > > > > > > > > > > their
> > > > > > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > > > > > > epoch-to-offset vectors? Or
> try
> > > to
> > > > > > > reconstruct
> > > > > > > > > > > its
> > > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > > > lineage based on the data
> > > remotely
> > > > > > > available?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > >
> > > > >
> > >
> >
> https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55,
> > > Satish
> > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > > > > > > > Please let us know if you
> > have
> > > any
> > > > > > > comments
> > > > > > > > > > on
> > > > > > > > > > > > > > > > > "transactional
> > > > > > > > > > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > > > > > > > > > and "follower
> > > requests/replication"
> > > > > > > mentioned
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > wiki.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25
> > PM
> > > > > Satish
> > > > > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > Thanks Jun for your
> > comments.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > >100. It would be useful
> to
> > > > > provide
> > > > > > > more
> > > > > > > > > > > > details
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > > > > > > are used. Otherwise, it's kind
> of
> > > hard
> > > > > to
> > > > > > > really
> > > > > > > > > > > > assess
> > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > > > > > > > apis are sufficient/redundant.
> A
> > > few
> > > > > > > examples
> > > > > > > > > > > below.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > We will update the wiki
> and
> > > let
> > > > > you
> > > > > > > know.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > >100.1 deleteRecords
> seems
> > to
> > > > > only
> > > > > > > advance
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > Log. How does that trigger the
> > > > > deletion of
> > > > > > > remote
> > > > > > > > > > > log
> > > > > > > > > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > RLMTask for leader
> > partition
> > > > > > > periodically
> > > > > > > > > > > > checks
> > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > > > remote log segments
> earlier
> > > to
> > > > > > > > > > logStartOffset
> > > > > > > > > > > > > and the
> > > > > > > > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > > > > > > > > remote log segment
> metadata
> > > and
> > > > > data
> > > > > > > are
> > > > > > > > > > > > deleted
> > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > > > > RLMM
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > >100.2 stopReplica with
> > > deletion
> > > > > is
> > > > > > > used
> > > > > > > > > > in 2
> > > > > > > > > > > > > cases
> > > > > > > > > > > > > > > (a)
> > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > > reassignment; (b) topic
> deletion.
> > > We
> > > > > only
> > > > > > > want to
> > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > > > > > > metadata in the second case.
> > Also,
> > > in
> > > > > the
> > > > > > > second
> > > > > > > > > > > > case,
> > > > > > > > > > > > > who
> > > > > > > > > > >
> > > > >
> > >
> > >
> >
>

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

Posted by Harsha Ch <ha...@gmail.com>.
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 <ju...@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 <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/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
> > 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' 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
> >
> > 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
> > > 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' 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
> > >
> > > 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 <ju...@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>
> > > > > 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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > > >
> > > > > > Satish.
> > > > > >
> > > > > >
> > > > > > On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana <
> > > > satish.duggana@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > > > >
> > > > > > >
> > > > > > > Satish.
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng
> > <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 <ju...@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
> > > > <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
> > > > > > > > > > unclean-leader-election.
> > > > > > > > > > 2) Since the user set local retention time (or local
> > retention
> > > > > > bytes), I
> > > > > > > > > > think we are expected to
> > > > > > > > > > keep that much local data when possible (avoid truncating
> > all
> > > > the
> > > > > > local
> > > > > > > > > > data). But, as you said,
> > > > > > > > > > unclean leader elections are very rare, this may not be a
> > big
> > > > > > problem.
> > > > > > > > > >
> > > > > > > > > > The current design uses the leader broker as
> > source-of-truth.
> > > > This
> > > > > > is
> > > > > > > > > > consistent with the
> > > > > > > > > > existing Kafka behavior.
> > > > > > > > > >
> > > > > > > > > > By using remote storage as the source-of-truth, the
> > follower
> > > > logic
> > > > > > can
> > > > > > > > > be a
> > > > > > > > > > little simpler,
> > > > > > > > > > but the leader logic is going to be more complex.
> Overall,
> > I
> > > > don't
> > > > > > see
> > > > > > > > > > there many benefits
> > > > > > > > > > of using remote storage as the source-of-truth.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <
> jun@confluent.io
> > >
> > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi, Satish,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the reply.
> > > > > > > > > > >
> > > > > > > > > > > 1001. In your example, I was thinking that you could
> just
> > > > > > download the
> > > > > > > > > > > latest leader epoch from the object store. After that
> you
> > > > know
> > > > > > the
> > > > > > > > > leader
> > > > > > > > > > > should end with offset 1100. The leader will delete all
> > its
> > > > > > local data
> > > > > > > > > > > before offset 1000 and start accepting new messages at
> > offset
> > > > > > 1100.
> > > > > > > > > > > Consumer requests for messages before offset 1100 will
> be
> > > > served
> > > > > > from
> > > > > > > > > the
> > > > > > > > > > > object store. The benefit with this approach is that
> it's
> > > > > > simpler to
> > > > > > > > > > reason
> > > > > > > > > > > about who is the source of truth. The downside is
> > slightly
> > > > > > increased
> > > > > > > > > > > unavailability window during unclean leader election.
> > Since
> > > > > > unclean
> > > > > > > > > > leader
> > > > > > > > > > > elections are rare, I am not sure if this is a big
> > concern.
> > > > > > > > > > >
> > > > > > > > > > > 1008. Yes, I think introducing sth like
> > local.retention.ms
> > > > > > seems more
> > > > > > > > > > > consistent.
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > > > > > > > > satish.duggana@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > HI Jun,
> > > > > > > > > > > > Thanks for your comments. We put our inline replies
> > below.
> > > > > > > > > > > >
> > > > > > > > > > > > 1001. I was thinking that you could just use the
> tiered
> > > > > > metadata to
> > > > > > > > > do
> > > > > > > > > > > the
> > > > > > > > > > > > reconciliation. The tiered metadata contains offset
> > ranges
> > > > and
> > > > > > epoch
> > > > > > > > > > > > history. Those should be enough for reconciliation
> > > > purposes.
> > > > > > > > > > > >
> > > > > > > > > > > > If we use remote storage as the source-of-truth
> during
> > > > > > > > > > > > unclean-leader-election, it's possible that after
> > > > > > reconciliation the
> > > > > > > > > > > > remote storage will have more recent data than the
> new
> > > > > > leader's local
> > > > > > > > > > > > storage. For example, the new leader's latest message
> > is
> > > > > > offset 1000,
> > > > > > > > > > > > while the remote storage has message 1100. In such a
> > case,
> > > > the
> > > > > > new
> > > > > > > > > > > > leader will have to download the messages from 1001
> to
> > > > 1100,
> > > > > > before
> > > > > > > > > > > > accepting new messages from producers. Otherwise,
> there
> > > > would
> > > > > > be a
> > > > > > > > > gap
> > > > > > > > > > > > in the local data between 1000 and 1101.
> > > > > > > > > > > >
> > > > > > > > > > > > Moreover, with the current design, leader epoch
> > history is
> > > > > > stored in
> > > > > > > > > > > > remote storage, rather than the metadata topic. We
> did
> > > > consider
> > > > > > > > > saving
> > > > > > > > > > > > epoch history in remote segment metadata. But the
> > concern
> > > > is
> > > > > > that
> > > > > > > > > > > > there is currently no limit for the epoch history
> size.
> > > > > > > > > Theoretically,
> > > > > > > > > > > > if a user has a very long remote retention time and
> > there
> > > > are
> > > > > > very
> > > > > > > > > > > > frequent leadership changes, the leader epoch history
> > can
> > > > > > become too
> > > > > > > > > > > > long to fit into a regular Kafka message.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 1003.3 Having just a serverEndpoint string is
> probably
> > not
> > > > > > enough.
> > > > > > > > > > > > Connecting to a Kafka cluster may need various
> security
> > > > > > credentials.
> > > > > > > > > We
> > > > > > > > > > > can
> > > > > > > > > > > > make RLMM configurable and pass in the properties
> > through
> > > > the
> > > > > > > > > > configure()
> > > > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > > > >
> > > > > > > > > > > > RLMM and  RSM are already configurable and they take
> > > > > > properties which
> > > > > > > > > > > > start with "remote.log.metadata." and
> > "remote.log.storage."
> > > > > > > > > > > > respectively and a few others. We have listener-name
> > as the
> > > > > > config
> > > > > > > > > for
> > > > > > > > > > > > RLMM and other properties(like security) can be sent
> > as you
> > > > > > > > > suggested.
> > > > > > > > > > > > We will update the KIP with the details.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > > log.retention.minutes,
> > > > > > > > > > and
> > > > > > > > > > > > added log.retention.ms later. If we are adding a new
> > > > > > configuration,
> > > > > > > > > ms
> > > > > > > > > > > > level config alone is enough and is simpler. We can
> > build
> > > > > > tools to
> > > > > > > > > make
> > > > > > > > > > > the
> > > > > > > > > > > > configuration at different granularities easier. The
> > > > > > definition of
> > > > > > > > > > > > log.retention.ms is "The number of milliseconds to
> > keep a
> > > > log
> > > > > > file
> > > > > > > > > > > before
> > > > > > > > > > > > deleting it". The deletion is independent of whether
> > > > tiering is
> > > > > > > > > enabled
> > > > > > > > > > > or
> > > > > > > > > > > > not. If this changes to just the local portion of the
> > > > data, we
> > > > > > are
> > > > > > > > > > > changing
> > > > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > > > >
> > > > > > > > > > > > We are fine with either way. We can go with
> > > > log.retention.xxxx
> > > > > > as the
> > > > > > > > > > > > effective log retention instead of local log
> retention.
> > > > With
> > > > > > this
> > > > > > > > > > > > convention, we need to introduce  local.log.retention
> > > > instead
> > > > > > of
> > > > > > > > > > > > remote.log.retention.ms that we proposed. If
> > > > log.retention.ms
> > > > > > as -1
> > > > > > > > > > > > then remote retention is also considered as unlimited
> > but
> > > > user
> > > > > > should
> > > > > > > > > > > > be able to set the local.retention.ms.
> > > > > > > > > > > > So, we need to introduce local.log.retention.ms and
> > > > > > > > > > > > local.log.retention.bytes which should  always  be <=
> > > > > > > > > > > > log.retention.ms/bytes respectively.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <
> > jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the reply. A few quick comments below.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1001. I was thinking that you could just use the
> > tiered
> > > > > > metadata to
> > > > > > > > > > do
> > > > > > > > > > > > the
> > > > > > > > > > > > > reconciliation. The tiered metadata contains offset
> > > > ranges
> > > > > > and
> > > > > > > > > epoch
> > > > > > > > > > > > > history. Those should be enough for reconciliation
> > > > purposes.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1003.3 Having just a serverEndpoint string is
> > probably
> > > > not
> > > > > > enough.
> > > > > > > > > > > > > Connecting to a Kafka cluster may need various
> > security
> > > > > > > > > credentials.
> > > > > > > > > > We
> > > > > > > > > > > > can
> > > > > > > > > > > > > make RLMM configurable and pass in the properties
> > > > through the
> > > > > > > > > > > configure()
> > > > > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > > > > > log.retention.minutes,
> > > > > > > > > > > and
> > > > > > > > > > > > > added log.retention.ms later. If we are adding a
> new
> > > > > > > > > configuration,
> > > > > > > > > > ms
> > > > > > > > > > > > > level config alone is enough and is simpler. We can
> > build
> > > > > > tools to
> > > > > > > > > > make
> > > > > > > > > > > > the
> > > > > > > > > > > > > configuration at different granularities easier.
> The
> > > > > > definition of
> > > > > > > > > > > > > log.retention.ms is "The number of milliseconds to
> > keep
> > > > a
> > > > > > log file
> > > > > > > > > > > > before
> > > > > > > > > > > > > deleting it". The deletion is independent of
> whether
> > > > tiering
> > > > > > is
> > > > > > > > > > enabled
> > > > > > > > > > > > or
> > > > > > > > > > > > > not. If this changes to just the local portion of
> the
> > > > data,
> > > > > > we are
> > > > > > > > > > > > changing
> > > > > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thank you for the comments! Ying, Harsha and I
> > > > discussed
> > > > > > and put
> > > > > > > > > > our
> > > > > > > > > > > > > > comments below.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1001. The KIP described a few scenarios of
> unclean
> > > > leader
> > > > > > > > > > elections.
> > > > > > > > > > > > This
> > > > > > > > > > > > > > is very useful, but I am wondering if this is the
> > best
> > > > > > approach.
> > > > > > > > > My
> > > > > > > > > > > > > > understanding of the proposed approach is to
> allow
> > the
> > > > new
> > > > > > > > > > (unclean)
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > to take new messages immediately. While this
> > increases
> > > > > > > > > > availability,
> > > > > > > > > > > it
> > > > > > > > > > > > > > creates the problem that there could be multiple
> > > > > > conflicting
> > > > > > > > > > segments
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the remote store for the same offset range. This
> > seems
> > > > to
> > > > > > make it
> > > > > > > > > > > > harder
> > > > > > > > > > > > > > for RLMM to determine which archived log segments
> > > > contain
> > > > > > the
> > > > > > > > > > correct
> > > > > > > > > > > > data.
> > > > > > > > > > > > > > For example, an archived log segment could at one
> > time
> > > > be
> > > > > > the
> > > > > > > > > > correct
> > > > > > > > > > > > data,
> > > > > > > > > > > > > > but be changed to incorrect data after an unclean
> > > > leader
> > > > > > > > > election.
> > > > > > > > > > An
> > > > > > > > > > > > > > alternative approach is to let the unclean leader
> > use
> > > > the
> > > > > > > > > archived
> > > > > > > > > > > > data as
> > > > > > > > > > > > > > the source of truth. So, when the new (unclean)
> > leader
> > > > > > takes
> > > > > > > > > over,
> > > > > > > > > > it
> > > > > > > > > > > > first
> > > > > > > > > > > > > > reconciles the local data based on the archived
> > data
> > > > before
> > > > > > > > > taking
> > > > > > > > > > > new
> > > > > > > > > > > > > > messages. This makes the job of RLMM a bit easier
> > > > since all
> > > > > > > > > > archived
> > > > > > > > > > > > data
> > > > > > > > > > > > > > are considered correct. This increases
> > availability a
> > > > bit.
> > > > > > > > > However,
> > > > > > > > > > > > since
> > > > > > > > > > > > > > unclean leader elections are rare, this may be
> ok.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Firstly, We don't want to assume the remote
> > storage is
> > > > more
> > > > > > > > > > reliable
> > > > > > > > > > > > than
> > > > > > > > > > > > > > Kafka. Kafka unclean leader election usually
> > happens
> > > > when
> > > > > > there
> > > > > > > > > is
> > > > > > > > > > a
> > > > > > > > > > > > large
> > > > > > > > > > > > > > scale outage that impacts multiple racks (or even
> > > > multiple
> > > > > > > > > > > availability
> > > > > > > > > > > > > > zones). In such a case, the remote storage may be
> > > > > > unavailable or
> > > > > > > > > > > > unstable.
> > > > > > > > > > > > > > Pulling a large amount of data from the remote
> > storage
> > > > to
> > > > > > > > > reconcile
> > > > > > > > > > > the
> > > > > > > > > > > > > > local data may also exacerbate the outage. With
> the
> > > > current
> > > > > > > > > design,
> > > > > > > > > > > > the new
> > > > > > > > > > > > > > leader can start working even when the remote
> > storage
> > > > is
> > > > > > > > > > temporarily
> > > > > > > > > > > > > > unavailable.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Secondly, it is not easier to implement the
> > reconciling
> > > > > > logic at
> > > > > > > > > > the
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > side. It can take a long time for the new leader
> to
> > > > > > download the
> > > > > > > > > > > remote
> > > > > > > > > > > > > > data and rebuild local producer id / leader epoch
> > > > > > information.
> > > > > > > > > > During
> > > > > > > > > > > > this
> > > > > > > > > > > > > > period, the leader cannot accept any requests
> from
> > the
> > > > > > clients
> > > > > > > > > and
> > > > > > > > > > > > > > followers. We have to introduce a new state for
> the
> > > > > > leader, and a
> > > > > > > > > > new
> > > > > > > > > > > > error
> > > > > > > > > > > > > > code to let the clients / followers know what is
> > > > happening.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > > > > > RemoteStorageManager.
> > > > > > > > > > > > We
> > > > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For
> > all
> > > > other
> > > > > > > > > methods,
> > > > > > > > > > > we
> > > > > > > > > > > > pass
> > > > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Nice catch, we can have the
> > RemoteLogSegmentMetadata
> > > > for
> > > > > > > > > > > copyLogSegment
> > > > > > > > > > > > > > too.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > > > inclusive
> > > > > > or
> > > > > > > > > > > exclusive?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > It is inclusive.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > > > leaderEpoch
> > > > > > > > > history
> > > > > > > > > > > for
> > > > > > > > > > > > a
> > > > > > > > > > > > > > partition.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Yes, updated the KIP with the new method.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1002.4 Could you define the type of
> > > > > > RemoteLogSegmentContext?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > This is removed in the latest code and it is not
> > > > needed.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1003.1 I am not sure why we need both of the
> > following
> > > > > > methods
> > > > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine
> them
> > into
> > > > > > one that
> > > > > > > > > > > takes
> > > > > > > > > > > > in
> > > > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > > > >     RemoteLogSegmentId
> > > > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Good point, these can be merged for now. I guess
> we
> > > > needed
> > > > > > them
> > > > > > > > > in
> > > > > > > > > > > > earlier
> > > > > > > > > > > > > > version of the implementation but it is not
> needed
> > now.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1003.2 There seems to be some inconsistencies in
> > the
> > > > > > methods
> > > > > > > > > > below. I
> > > > > > > > > > > > am
> > > > > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata
> > and the
> > > > > > other
> > > > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > > > >     void
> > > > putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > > > >     void
> > > > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > RLMM stores RemoteLogSegmentMetadata which is
> > > > identified by
> > > > > > > > > > > > > > RemoteLogsSegmentId. So, when it is added it
> takes
> > > > > > > > > > > > > > RemoteLogSegmentMetadata. `delete` operation
> needs
> > only
> > > > > > > > > > > > RemoteLogsSegmentId
> > > > > > > > > > > > > > as RemoteLogSegmentMetadata can be identified
> with
> > > > > > > > > > > RemoteLogsSegmentId.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > > > > serverEndpoint), what
> > > > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > This can be used by RLMM implementation to
> connect
> > to
> > > > the
> > > > > > local
> > > > > > > > > > Kafka
> > > > > > > > > > > > > > cluster. Incase of default  implementation, it is
> > used
> > > > in
> > > > > > > > > > > initializing
> > > > > > > > > > > > > > kafka clients connecting to the local cluster.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1004. It would be useful to document how all the
> > new
> > > > APIs
> > > > > > are
> > > > > > > > > being
> > > > > > > > > > > > used.
> > > > > > > > > > > > > > For example, when is
> > > > > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > > > being
> > > > > > > > > > > > set
> > > > > > > > > > > > > > and used? How are
> > > > > > > > > > > > > >
> > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > > > > > > > > > used?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > RLMM APIs are going through the changes and they
> > > > should be
> > > > > > ready
> > > > > > > > > > in a
> > > > > > > > > > > > few
> > > > > > > > > > > > > > days. I will update the KIP and the mail  thread
> > once
> > > > they
> > > > > > are
> > > > > > > > > > ready.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1005. Handling partition deletion: The KIP says
> > "RLMM
> > > > will
> > > > > > > > > > eventually
> > > > > > > > > > > > > > delete these segments by using
> > RemoteStorageManager."
> > > > Which
> > > > > > > > > replica
> > > > > > > > > > > > does
> > > > > > > > > > > > > > this logic?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > This is a good point. When a topic is deleted, it
> > will
> > > > not
> > > > > > have
> > > > > > > > > any
> > > > > > > > > > > > > > leader/followers to do the cleanup. We will have
> a
> > > > cleaner
> > > > > > agent
> > > > > > > > > > on a
> > > > > > > > > > > > > > single broker in the cluster to do this cleanup,
> we
> > > > plan
> > > > > > to add
> > > > > > > > > > that
> > > > > > > > > > > in
> > > > > > > > > > > > > > controller broker.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1006. "If there are any failures in removing
> > remote log
> > > > > > segments
> > > > > > > > > > then
> > > > > > > > > > > > those
> > > > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > > > remote-log-segment-id)
> > > > > > > > > > > > from
> > > > > > > > > > > > > > that topic and clean them up from remote storage.
> > "
> > > > Not
> > > > > > sure if
> > > > > > > > > > it's
> > > > > > > > > > > > worth
> > > > > > > > > > > > > > the complexity of adding another topic. Could we
> > just
> > > > > > retry?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Sure, we can keep this simpler for now by logging
> > an
> > > > error
> > > > > > after
> > > > > > > > > > > > retries.
> > > > > > > > > > > > > > We can give users a better way to process this in
> > > > future.
> > > > > > Oneway
> > > > > > > > > > can
> > > > > > > > > > > > be a
> > > > > > > > > > > > > > dead letter topic which can be configured by the
> > user.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse
> the
> > > > > > existing
> > > > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We have 2 types of delayed operations waiting
> for 2
> > > > > > different
> > > > > > > > > > events.
> > > > > > > > > > > > > > DelayedFetch waits for new messages from
> producers.
> > > > > > > > > > > DelayedRemoteFetch
> > > > > > > > > > > > > > waits for the remote-storage-read-task to finish.
> > When
> > > > > > either of
> > > > > > > > > > the
> > > > > > > > > > > 2
> > > > > > > > > > > > > > events happens, we only want to notify one type
> of
> > the
> > > > > > delayed
> > > > > > > > > > > > operations.
> > > > > > > > > > > > > > It would be inefficient to put 2 types of delayed
> > > > > > operations in
> > > > > > > > > one
> > > > > > > > > > > > > > purgatory, as the tryComplete() methods of the
> > delayed
> > > > > > operations
> > > > > > > > > > can
> > > > > > > > > > > > be
> > > > > > > > > > > > > > triggered by irrelevant events.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > > > > remote.log.retention.minutes,
> > > > > > > > > > > > > > remote.log.retention.hours: It seems that we just
> > need
> > > > the
> > > > > > ms
> > > > > > > > > one.
> > > > > > > > > > > > Also,
> > > > > > > > > > > > > > are we changing the meaning of existing config
> > > > > > log.retention.ms
> > > > > > > > > to
> > > > > > > > > > > > mean
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > local retention? For backward compatibility, it's
> > > > better
> > > > > > to not
> > > > > > > > > > > change
> > > > > > > > > > > > the
> > > > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We agree that we only need
> remote.log.retention.ms
> > .
> > > > But,
> > > > > > the
> > > > > > > > > > > existing
> > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > configuration
> > > > > > > > > > > > > > has 3 properties (log.retention.ms,
> > > > log.retention.minutes,
> > > > > > > > > > > > > > log.retention.hours). We just
> > > > > > > > > > > > > > want to keep consistent with the existing
> > properties.
> > > > > > > > > > > > > > Existing log.retention.xxxx config is about log
> > > > retention
> > > > > > in
> > > > > > > > > > broker’s
> > > > > > > > > > > > > > storage which is local. It should be easy for
> > users to
> > > > > > configure
> > > > > > > > > > > > partition
> > > > > > > > > > > > > > storage with local retention and remote retention
> > > > based on
> > > > > > their
> > > > > > > > > > > usage.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the
> > topic
> > > > > > level?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > We can introduce topic level config for the same
> > > > remote.log
> > > > > > > > > > settings.
> > > > > > > > > > > > User
> > > > > > > > > > > > > > can set the desired config while creating the
> > topic.
> > > > > > > > > > > > > > remote.log.storage.enable property is not allowed
> > to be
> > > > > > updated
> > > > > > > > > > after
> > > > > > > > > > > > the
> > > > > > > > > > > > > > topic is created. Other remote.log.* properties
> > can be
> > > > > > modified.
> > > > > > > > > We
> > > > > > > > > > > > will
> > > > > > > > > > > > > > support flipping remote.log.storage.enable in
> next
> > > > > > versions.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1009. It would be useful to list all limitations
> > in a
> > > > > > separate
> > > > > > > > > > > section:
> > > > > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a
> > topic
> > > > from
> > > > > > delete
> > > > > > > > > > to
> > > > > > > > > > > > > > compact and vice versa allowed when tiering is
> > enabled?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 to have limitations in a separate section. We
> > will
> > > > > > update the
> > > > > > > > > > KIP
> > > > > > > > > > > > with
> > > > > > > > > > > > > > that.
> > > > > > > > > > > > > > Topic  created with effective value for
> > > > remote.log.enabled
> > > > > > as
> > > > > > > > > true,
> > > > > > > > > > > > can not
> > > > > > > > > > > > > > change its retention policy from delete to
> compact.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1010. Thanks for performance numbers. Are those
> > with
> > > > > > RocksDB as
> > > > > > > > > the
> > > > > > > > > > > > cache?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > No, We have not yet added RocksDB support. This
> is
> > > > based on
> > > > > > > > > > in-memory
> > > > > > > > > > > > map
> > > > > > > > > > > > > > representation. We will add that support and
> update
> > > > this
> > > > > > thread
> > > > > > > > > > after
> > > > > > > > > > > > > > updating the KIP with the numbers.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <
> > > > jun@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > > > below.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1000. Regarding Colin's question on querying
> the
> > > > metadata
> > > > > > > > > > directly
> > > > > > > > > > > > in the
> > > > > > > > > > > > > > > remote block store. One issue is that not all
> > block
> > > > > > stores
> > > > > > > > > offer
> > > > > > > > > > > the
> > > > > > > > > > > > > > needed
> > > > > > > > > > > > > > > api to query the metadata. For example, S3 only
> > > > offers
> > > > > > an api
> > > > > > > > > to
> > > > > > > > > > > list
> > > > > > > > > > > > > > > objects under a prefix and this api has the
> > eventual
> > > > > > > > > consistency
> > > > > > > > > > > > > > semantic.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1001. The KIP described a few scenarios of
> > unclean
> > > > leader
> > > > > > > > > > > elections.
> > > > > > > > > > > > This
> > > > > > > > > > > > > > > is very useful, but I am wondering if this is
> the
> > > > best
> > > > > > > > > approach.
> > > > > > > > > > My
> > > > > > > > > > > > > > > understanding of the proposed approach is to
> > allow
> > > > the
> > > > > > new
> > > > > > > > > > > (unclean)
> > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > to take new messages immediately. While this
> > > > increases
> > > > > > > > > > > availability,
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > creates the problem that there could be
> multiple
> > > > > > conflicting
> > > > > > > > > > > > segments in
> > > > > > > > > > > > > > > the remote store for the same offset range.
> This
> > > > seems
> > > > > > to make
> > > > > > > > > it
> > > > > > > > > > > > harder
> > > > > > > > > > > > > > > for RLMM to determine which archived log
> segments
> > > > > > contain the
> > > > > > > > > > > correct
> > > > > > > > > > > > > > data.
> > > > > > > > > > > > > > > For example, an archived log segment could at
> one
> > > > time
> > > > > > be the
> > > > > > > > > > > correct
> > > > > > > > > > > > > > data,
> > > > > > > > > > > > > > > but be changed to incorrect data after an
> unclean
> > > > leader
> > > > > > > > > > election.
> > > > > > > > > > > An
> > > > > > > > > > > > > > > alternative approach is to let the unclean
> leader
> > > > use the
> > > > > > > > > > archived
> > > > > > > > > > > > data
> > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > the source of truth. So, when the new (unclean)
> > > > leader
> > > > > > takes
> > > > > > > > > > over,
> > > > > > > > > > > it
> > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > reconciles the local data based on the archived
> > data
> > > > > > before
> > > > > > > > > > taking
> > > > > > > > > > > > new
> > > > > > > > > > > > > > > messages. This makes the job of RLMM a bit
> easier
> > > > since
> > > > > > all
> > > > > > > > > > > archived
> > > > > > > > > > > > data
> > > > > > > > > > > > > > > are considered correct. This increases
> > availability a
> > > > > > bit.
> > > > > > > > > > However,
> > > > > > > > > > > > since
> > > > > > > > > > > > > > > unclean leader elections are rare, this may be
> > ok.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > > > > 1002.1 There seems to be some inconsistencies
> in
> > > > > > > > > > > > RemoteStorageManager. We
> > > > > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment().
> For
> > all
> > > > > > other
> > > > > > > > > > methods,
> > > > > > > > > > > > we
> > > > > > > > > > > > > > pass
> > > > > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > > > > > inclusive or
> > > > > > > > > > > > exclusive?
> > > > > > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > > > > > leaderEpoch
> > > > > > > > > > history
> > > > > > > > > > > > for a
> > > > > > > > > > > > > > > partition.
> > > > > > > > > > > > > > > 1002.4 Could you define the type of
> > > > > > RemoteLogSegmentContext?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > > > > > 1003.1 I am not sure why we need both of the
> > > > following
> > > > > > methods
> > > > > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine
> > them
> > > > into
> > > > > > one
> > > > > > > > > that
> > > > > > > > > > > > takes in
> > > > > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > > > > >     RemoteLogSegmentId
> > > > > > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > > > 1003.2 There seems to be some inconsistencies
> in
> > the
> > > > > > methods
> > > > > > > > > > below.
> > > > > > > > > > > > I am
> > > > > > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata
> > and
> > > > the
> > > > > > other
> > > > > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > > > > >     void
> > > > putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > > > > >     void
> > > > > > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > > > > serverEndpoint),
> > > > > > > > > what
> > > > > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1004. It would be useful to document how all
> the
> > new
> > > > > > APIs are
> > > > > > > > > > being
> > > > > > > > > > > > used.
> > > > > > > > > > > > > > > For example, when is
> > > > > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > > > > > being set
> > > > > > > > > > > > > > > and used? How are
> > > > > > > > > > > > > > >
> > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > > > > > > > > being
> > > > > > > > > > > > used?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1005. Handling partition deletion: The KIP says
> > "RLMM
> > > > > > will
> > > > > > > > > > > eventually
> > > > > > > > > > > > > > > delete these segments by using
> > RemoteStorageManager."
> > > > > > Which
> > > > > > > > > > replica
> > > > > > > > > > > > does
> > > > > > > > > > > > > > > this logic?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1006. "If there are any failures in removing
> > remote
> > > > log
> > > > > > > > > segments
> > > > > > > > > > > then
> > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > > > > remote-log-segment-id)
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > that topic and clean them up from remote
> > storage.  "
> > > > Not
> > > > > > sure
> > > > > > > > > if
> > > > > > > > > > > it's
> > > > > > > > > > > > > > worth
> > > > > > > > > > > > > > > the complexity of adding another topic. Could
> we
> > just
> > > > > > retry?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse
> > the
> > > > > > existing
> > > > > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > > > > remote.log.retention.minutes,
> > > > > > > > > > > > > > > remote.log.retention.hours: It seems that we
> just
> > > > need
> > > > > > the ms
> > > > > > > > > > one.
> > > > > > > > > > > > Also,
> > > > > > > > > > > > > > > are we changing the meaning of existing config
> > > > > > > > > log.retention.ms
> > > > > > > > > > to
> > > > > > > > > > > > mean
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > local retention? For backward compatibility,
> it's
> > > > better
> > > > > > to not
> > > > > > > > > > > > change
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at
> the
> > > > topic
> > > > > > level?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1009. It would be useful to list all
> limitations
> > in a
> > > > > > separate
> > > > > > > > > > > > section:
> > > > > > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a
> > topic
> > > > > > from
> > > > > > > > > delete
> > > > > > > > > > > to
> > > > > > > > > > > > > > > compact and vice versa allowed when tiering is
> > > > enabled?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1010. Thanks for performance numbers. Are those
> > with
> > > > > > RocksDB as
> > > > > > > > > > the
> > > > > > > > > > > > > > cache?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <
> > > > > > harsha.ch@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > >                Thats not what we said in the
> > > > previous
> > > > > > email.
> > > > > > > > > > RLMM
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > pluggable storage and by running numbers even
> > 1PB
> > > > data
> > > > > > you do
> > > > > > > > > > not
> > > > > > > > > > > > need
> > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > than 10GB local storage.
> > > > > > > > > > > > > > > > If in future this becomes a blocker for any
> > users
> > > > we
> > > > > > can
> > > > > > > > > > revisit
> > > > > > > > > > > > but
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > does not warrant another implementation at
> this
> > > > point
> > > > > > to push
> > > > > > > > > > the
> > > > > > > > > > > > data
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > We can ofcourse implement another RLMM that
> is
> > > > > > optional for
> > > > > > > > > > users
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > configure to push to remote. But that doesn't
> > need
> > > > to
> > > > > > be
> > > > > > > > > > > addressed
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Harsha
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe
> <
> > > > > > > > > > cmccabe@apache.org
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi Ying,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks for the response.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > It sounds like you agree that storing the
> > > > metadata
> > > > > > in the
> > > > > > > > > > > remote
> > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > would be a better design overall.  Given
> that
> > > > that's
> > > > > > true,
> > > > > > > > > is
> > > > > > > > > > > > there
> > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > reason to include the worse implementation
> > based
> > > > on
> > > > > > > > > RocksDB?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Choosing a long-term metadata store is not
> > > > something
> > > > > > that
> > > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > lightly.  It can take users years to
> migrate
> > from
> > > > > > metadata
> > > > > > > > > > > store
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > other.  I also don't think it's realistic
> or
> > > > > > desirable for
> > > > > > > > > > > users
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > write
> > > > > > > > > > > > > > > > > their own metadata stores.  Even assuming
> > that
> > > > they
> > > > > > could
> > > > > > > > > do
> > > > > > > > > > a
> > > > > > > > > > > > good
> > > > > > > > > > > > > > job
> > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > this, it would create huge fragmentation in
> > the
> > > > Kafka
> > > > > > > > > > > ecosystem.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng
> > wrote:
> > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Satish and I are still discussing some
> > details
> > > > > > about how
> > > > > > > > > to
> > > > > > > > > > > > handle
> > > > > > > > > > > > > > > > > > transactions / producer ids. Satish is
> > going to
> > > > > > make some
> > > > > > > > > > > minor
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > RLMM API and other parts. Other than
> that,
> > we
> > > > have
> > > > > > > > > finished
> > > > > > > > > > > > > > updating
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I agree with Colin that the current
> design
> > of
> > > > using
> > > > > > > > > rocksDB
> > > > > > > > > > > is
> > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > optimal. But this design is simple and
> > should
> > > > work
> > > > > > for
> > > > > > > > > > almost
> > > > > > > > > > > > all
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > existing Kafka users. RLMM is a plugin.
> > Users
> > > > can
> > > > > > replace
> > > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > their own RLMM implementation, if needed.
> > So, I
> > > > > > think we
> > > > > > > > > > can
> > > > > > > > > > > > keep
> > > > > > > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > > > > > for now. What do you think?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Ying
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks for the update. It's good to see
> > the
> > > > > > progress on
> > > > > > > > > > > this.
> > > > > > > > > > > > > > > Please
> > > > > > > > > > > > > > > > > let us
> > > > > > > > > > > > > > > > > > > know when you are done updating the KIP
> > wiki.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying
> > Zheng
> > > > > > > > > > > > > > <yingz@uber.com.invalid
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Satish and I have added more design
> > > > details in
> > > > > > the
> > > > > > > > > KIP,
> > > > > > > > > > > > > > including
> > > > > > > > > > > > > > > > > how to
> > > > > > > > > > > > > > > > > > > > keep consistency between replicas
> > > > (especially
> > > > > > when
> > > > > > > > > > there
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > > leadership
> > > > > > > > > > > > > > > > > > > > changes / log truncations) and new
> > > > metrics. We
> > > > > > also
> > > > > > > > > > made
> > > > > > > > > > > > some
> > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > minor
> > > > > > > > > > > > > > > > > > > > changes in the doc. We will finish
> the
> > KIP
> > > > > > changes in
> > > > > > > > > > the
> > > > > > > > > > > > next
> > > > > > > > > > > > > > > > > couple of
> > > > > > > > > > > > > > > > > > > > days. We will let you know when we
> are
> > > > done.
> > > > > > Most of
> > > > > > > > > > the
> > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > already updated to the wiki KIP. You
> > can
> > > > take
> > > > > > a look.
> > > > > > > > > > But
> > > > > > > > > > > > it's
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > As for the implementation, the code
> is
> > > > mostly
> > > > > > done
> > > > > > > > > and
> > > > > > > > > > we
> > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > had
> > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > feature tests / system tests. I have
> > added
> > > > the
> > > > > > > > > > > performance
> > > > > > > > > > > > test
> > > > > > > > > > > > > > > > > results
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > the KIP. However the recent design
> > changes
> > > > > > (e.g.
> > > > > > > > > leader
> > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > management / log truncation / some of
> > the
> > > > new
> > > > > > > > > metrics)
> > > > > > > > > > > > have not
> > > > > > > > > > > > > > > > been
> > > > > > > > > > > > > > > > > > > > implemented yet. It will take about 2
> > weeks
> > > > > > for us to
> > > > > > > > > > > > implement
> > > > > > > > > > > > > > > > > after you
> > > > > > > > > > > > > > > > > > > > review and agree with those design
> > changes.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun
> Rao
> > <
> > > > > > > > > > jun@confluent.io
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Any new updates on the KIP? This
> > feature
> > > > is
> > > > > > one of
> > > > > > > > > > the
> > > > > > > > > > > > most
> > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > most requested features in Apache
> > Kafka
> > > > > > right now.
> > > > > > > > > It
> > > > > > > > > > > > would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > helpful
> > > > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > > we can make sustained progress on
> > this.
> > > > > > Could you
> > > > > > > > > > share
> > > > > > > > > > > > how
> > > > > > > > > > > > > > far
> > > > > > > > > > > > > > > > > along
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > the design/implementation right
> now?
> > Is
> > > > there
> > > > > > > > > > anything
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > As for "transactional support" and
> > > > "follower
> > > > > > > > > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > > > > > > > > further comments from me as long as
> > the
> > > > > > producer
> > > > > > > > > > state
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > be restored properly from the
> object
> > > > store
> > > > > > when
> > > > > > > > > > needed.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM
> Satish
> > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > We did not want to add many
> > > > implementation
> > > > > > > > > details
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > But we
> > > > > > > > > > > > > > > > > > > > > > decided to add them in the KIP as
> > > > appendix
> > > > > > or
> > > > > > > > > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > > > > > > > > follower fetch protocol) to
> > describe
> > > > the
> > > > > > flow
> > > > > > > > > with
> > > > > > > > > > > the
> > > > > > > > > > > > main
> > > > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > > > > That will answer most of the
> > queries. I
> > > > > > will
> > > > > > > > > update
> > > > > > > > > > > on
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > > mail
> > > > > > > > > > > > > > > > > > > > > > thread when the respective
> > sections are
> > > > > > updated.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM
> > > > Alexandre
> > > > > > Dupriez
> > > > > > > > > > > > > > > > > > > > > > <al...@gmail.com>
> > wrote:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > A couple of questions specific
> > to the
> > > > > > section
> > > > > > > > > > > > "Follower
> > > > > > > > > > > > > > > > > > > > > > > Requests/Replication", pages
> > 16:17
> > > > in the
> > > > > > > > > design
> > > > > > > > > > > > document
> > > > > > > > > > > > > > > > [1].
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900. It is mentioned that
> > followers
> > > > fetch
> > > > > > > > > > auxiliary
> > > > > > > > > > > > > > states
> > > > > > > > > > > > > > > > > from the
> > > > > > > > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900.a Does the consistency
> model
> > of
> > > > the
> > > > > > > > > external
> > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > impacts
> > > > > > > > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > > > > > > > > of leader epochs and other
> > auxiliary
> > > > > > data?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900.b What are the benefits of
> > using
> > > > a
> > > > > > > > > mechanism
> > > > > > > > > > to
> > > > > > > > > > > > store
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > > > > > > > the leader epochs which is
> > different
> > > > > > from other
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > tiered segments? What are the
> > > > benefits of
> > > > > > > > > > > retrieving
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > > on-demand from the follower
> > rather
> > > > than
> > > > > > relying
> > > > > > > > > > on
> > > > > > > > > > > > > > > > propagation
> > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > topic __remote_log_metadata?
> What
> > > > are the
> > > > > > > > > > > advantages
> > > > > > > > > > > > over
> > > > > > > > > > > > > > > > > using a
> > > > > > > > > > > > > > > > > > > > > > > dedicated control structure
> > (e.g. a
> > > > new
> > > > > > record
> > > > > > > > > > > type)
> > > > > > > > > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > > > > > > > > this topic? Since in the
> > document,
> > > > > > different
> > > > > > > > > > > control
> > > > > > > > > > > > > > paths
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > operating in the system, how
> are
> > the
> > > > > > metadata
> > > > > > > > > > > stored
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > __remote_log_metadata [which
> also
> > > > > > include the
> > > > > > > > > > epoch
> > > > > > > > > > > > of
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > > which offloaded a segment] and
> > the
> > > > remote
> > > > > > > > > > auxiliary
> > > > > > > > > > > > > > states,
> > > > > > > > > > > > > > > > > kept in
> > > > > > > > > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900.c A follower can encounter
> an
> > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > > > > > > > > this in response to a Fetch or
> > > > > > > > > > OffsetForLeaderEpoch
> > > > > > > > > > > > > > > request?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900.d What happens if, after a
> > > > follower
> > > > > > > > > > encountered
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE
> > > > response,
> > > > > > its
> > > > > > > > > > > > attempts to
> > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > > leader epochs fail (for
> instance,
> > > > > > because the
> > > > > > > > > > > remote
> > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > temporarily unavailable)? Does
> > the
> > > > > > follower
> > > > > > > > > > > > fallbacks to
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > mode
> > > > > > > > > > > > > > > > > > > where
> > > > > > > > > > > > > > > > > > > > > > > it ignores tiered segments, and
> > > > applies
> > > > > > > > > > truncation
> > > > > > > > > > > > using
> > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > > > > > available information? What
> > happens
> > > > when
> > > > > > access
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > > > > is restored? How is the replica
> > > > lineage
> > > > > > > > > inferred
> > > > > > > > > > by
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > > epochs reconciled with the
> > follower's
> > > > > > replica
> > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > > evolved? Does the follower
> > remember
> > > > > > fetching
> > > > > > > > > > > > auxiliary
> > > > > > > > > > > > > > > states
> > > > > > > > > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > > > > > > > > in the past and attempt
> > > > reconciliation?
> > > > > > Is
> > > > > > > > > there
> > > > > > > > > > a
> > > > > > > > > > > > plan
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > offer
> > > > > > > > > > > > > > > > > > > > > > > different strategies in this
> > > > scenario,
> > > > > > > > > > configurable
> > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900.e Is the leader epoch cache
> > > > > > offloaded with
> > > > > > > > > > > every
> > > > > > > > > > > > > > > segment?
> > > > > > > > > > > > > > > > > Or
> > > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > > > a new checkpoint is detected?
> If
> > that
> > > > > > > > > information
> > > > > > > > > > > is
> > > > > > > > > > > > not
> > > > > > > > > > > > > > > > always
> > > > > > > > > > > > > > > > > > > > > > > offloaded to avoid duplicating
> > data,
> > > > how
> > > > > > does
> > > > > > > > > the
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > > > > satisfy the request to retrieve
> > it?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900.f Since the leader epoch
> > cache
> > > > > > covers the
> > > > > > > > > > > entire
> > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > > > > > > > > what happens if, after a leader
> > epoch
> > > > > > cache
> > > > > > > > > file
> > > > > > > > > > is
> > > > > > > > > > > > > > > offloaded
> > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > given segment, the local epoch
> > cache
> > > > is
> > > > > > > > > truncated
> > > > > > > > > > > > [not
> > > > > > > > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > a range of offset included in
> > tiered
> > > > > > segments]?
> > > > > > > > > > How
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > local leader epoch caches kept
> > > > > > consistent?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900.g Consumer can also use
> > leader
> > > > > > epochs (e.g.
> > > > > > > > > > to
> > > > > > > > > > > > enable
> > > > > > > > > > > > > > > > > fencing
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > protect against stale leaders).
> > What
> > > > > > > > > differences
> > > > > > > > > > > > would
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > between consumer and follower
> > > > fetches?
> > > > > > > > > > Especially,
> > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > > > > > > > also fetch leader epoch
> > information
> > > > from
> > > > > > the
> > > > > > > > > > remote
> > > > > > > > > > > > > > > storage?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 900.h Assume a newly elected
> > leader
> > > > of a
> > > > > > > > > > > > topic-partition
> > > > > > > > > > > > > > > > > detects
> > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > > recent segments are available
> in
> > the
> > > > > > external
> > > > > > > > > > > > storage,
> > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > epochs
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > its local epoch. Does it ignore
> > these
> > > > > > segments
> > > > > > > > > > and
> > > > > > > > > > > > their
> > > > > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > > > > > epoch-to-offset vectors? Or try
> > to
> > > > > > reconstruct
> > > > > > > > > > its
> > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > > lineage based on the data
> > remotely
> > > > > > available?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > >
> > > >
> >
> https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55,
> > Satish
> > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > > > > > > Please let us know if you
> have
> > any
> > > > > > comments
> > > > > > > > > on
> > > > > > > > > > > > > > > > "transactional
> > > > > > > > > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > > > > > > > > and "follower
> > requests/replication"
> > > > > > mentioned
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > wiki.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25
> PM
> > > > Satish
> > > > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Thanks Jun for your
> comments.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >100. It would be useful to
> > > > provide
> > > > > > more
> > > > > > > > > > > details
> > > > > > > > > > > > on
> > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > > > > > are used. Otherwise, it's kind of
> > hard
> > > > to
> > > > > > really
> > > > > > > > > > > assess
> > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > > > > > > apis are sufficient/redundant. A
> > few
> > > > > > examples
> > > > > > > > > > below.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > We will update the wiki and
> > let
> > > > you
> > > > > > know.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >100.1 deleteRecords seems
> to
> > > > only
> > > > > > advance
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > Log. How does that trigger the
> > > > deletion of
> > > > > > remote
> > > > > > > > > > log
> > > > > > > > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > RLMTask for leader
> partition
> > > > > > periodically
> > > > > > > > > > > checks
> > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > > remote log segments earlier
> > to
> > > > > > > > > logStartOffset
> > > > > > > > > > > > and the
> > > > > > > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > > > > > > > remote log segment metadata
> > and
> > > > data
> > > > > > are
> > > > > > > > > > > deleted
> > > > > > > > > > > > by
> > > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > > > RLMM
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >100.2 stopReplica with
> > deletion
> > > > is
> > > > > > used
> > > > > > > > > in 2
> > > > > > > > > > > > cases
> > > > > > > > > > > > > > (a)
> > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > reassignment; (b) topic deletion.
> > We
> > > > only
> > > > > > want to
> > > > > > > > > > > > delete
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > > > > > metadata in the second case.
> Also,
> > in
> > > > the
> > > > > > second
> > > > > > > > > > > case,
> > > > > > > > > > > > who
> > > > > > > > > >
> > > >
> >
> >
>

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

Posted by Jun Rao <ju...@confluent.io>.
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 <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
>
> https://cwiki.apache.org/confluence/display/KAFKA/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
> 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' 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 <kp...@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
> > 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' 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
> >
> > 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 <ju...@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>
> > > > 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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > >
> > > > > Satish.
> > > > >
> > > > >
> > > > > On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana <
> > > satish.duggana@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > > >
> > > > > >
> > > > > > Satish.
> > > > > >
> > > > > >
> > > > > > On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng
> <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 <ju...@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
> > > <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
> > > > > > > > > unclean-leader-election.
> > > > > > > > > 2) Since the user set local retention time (or local
> retention
> > > > > bytes), I
> > > > > > > > > think we are expected to
> > > > > > > > > keep that much local data when possible (avoid truncating
> all
> > > the
> > > > > local
> > > > > > > > > data). But, as you said,
> > > > > > > > > unclean leader elections are very rare, this may not be a
> big
> > > > > problem.
> > > > > > > > >
> > > > > > > > > The current design uses the leader broker as
> source-of-truth.
> > > This
> > > > > is
> > > > > > > > > consistent with the
> > > > > > > > > existing Kafka behavior.
> > > > > > > > >
> > > > > > > > > By using remote storage as the source-of-truth, the
> follower
> > > logic
> > > > > can
> > > > > > > > be a
> > > > > > > > > little simpler,
> > > > > > > > > but the leader logic is going to be more complex. Overall,
> I
> > > don't
> > > > > see
> > > > > > > > > there many benefits
> > > > > > > > > of using remote storage as the source-of-truth.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <jun@confluent.io
> >
> > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Satish,
> > > > > > > > > >
> > > > > > > > > > Thanks for the reply.
> > > > > > > > > >
> > > > > > > > > > 1001. In your example, I was thinking that you could just
> > > > > download the
> > > > > > > > > > latest leader epoch from the object store. After that you
> > > know
> > > > > the
> > > > > > > > leader
> > > > > > > > > > should end with offset 1100. The leader will delete all
> its
> > > > > local data
> > > > > > > > > > before offset 1000 and start accepting new messages at
> offset
> > > > > 1100.
> > > > > > > > > > Consumer requests for messages before offset 1100 will be
> > > served
> > > > > from
> > > > > > > > the
> > > > > > > > > > object store. The benefit with this approach is that it's
> > > > > simpler to
> > > > > > > > > reason
> > > > > > > > > > about who is the source of truth. The downside is
> slightly
> > > > > increased
> > > > > > > > > > unavailability window during unclean leader election.
> Since
> > > > > unclean
> > > > > > > > > leader
> > > > > > > > > > elections are rare, I am not sure if this is a big
> concern.
> > > > > > > > > >
> > > > > > > > > > 1008. Yes, I think introducing sth like
> local.retention.ms
> > > > > seems more
> > > > > > > > > > consistent.
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > > > > > > > satish.duggana@gmail.com
> > > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > HI Jun,
> > > > > > > > > > > Thanks for your comments. We put our inline replies
> below.
> > > > > > > > > > >
> > > > > > > > > > > 1001. I was thinking that you could just use the tiered
> > > > > metadata to
> > > > > > > > do
> > > > > > > > > > the
> > > > > > > > > > > reconciliation. The tiered metadata contains offset
> ranges
> > > and
> > > > > epoch
> > > > > > > > > > > history. Those should be enough for reconciliation
> > > purposes.
> > > > > > > > > > >
> > > > > > > > > > > If we use remote storage as the source-of-truth during
> > > > > > > > > > > unclean-leader-election, it's possible that after
> > > > > reconciliation the
> > > > > > > > > > > remote storage will have more recent data than the new
> > > > > leader's local
> > > > > > > > > > > storage. For example, the new leader's latest message
> is
> > > > > offset 1000,
> > > > > > > > > > > while the remote storage has message 1100. In such a
> case,
> > > the
> > > > > new
> > > > > > > > > > > leader will have to download the messages from 1001 to
> > > 1100,
> > > > > before
> > > > > > > > > > > accepting new messages from producers. Otherwise, there
> > > would
> > > > > be a
> > > > > > > > gap
> > > > > > > > > > > in the local data between 1000 and 1101.
> > > > > > > > > > >
> > > > > > > > > > > Moreover, with the current design, leader epoch
> history is
> > > > > stored in
> > > > > > > > > > > remote storage, rather than the metadata topic. We did
> > > consider
> > > > > > > > saving
> > > > > > > > > > > epoch history in remote segment metadata. But the
> concern
> > > is
> > > > > that
> > > > > > > > > > > there is currently no limit for the epoch history size.
> > > > > > > > Theoretically,
> > > > > > > > > > > if a user has a very long remote retention time and
> there
> > > are
> > > > > very
> > > > > > > > > > > frequent leadership changes, the leader epoch history
> can
> > > > > become too
> > > > > > > > > > > long to fit into a regular Kafka message.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 1003.3 Having just a serverEndpoint string is probably
> not
> > > > > enough.
> > > > > > > > > > > Connecting to a Kafka cluster may need various security
> > > > > credentials.
> > > > > > > > We
> > > > > > > > > > can
> > > > > > > > > > > make RLMM configurable and pass in the properties
> through
> > > the
> > > > > > > > > configure()
> > > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > > >
> > > > > > > > > > > RLMM and  RSM are already configurable and they take
> > > > > properties which
> > > > > > > > > > > start with "remote.log.metadata." and
> "remote.log.storage."
> > > > > > > > > > > respectively and a few others. We have listener-name
> as the
> > > > > config
> > > > > > > > for
> > > > > > > > > > > RLMM and other properties(like security) can be sent
> as you
> > > > > > > > suggested.
> > > > > > > > > > > We will update the KIP with the details.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > log.retention.minutes,
> > > > > > > > > and
> > > > > > > > > > > added log.retention.ms later. If we are adding a new
> > > > > configuration,
> > > > > > > > ms
> > > > > > > > > > > level config alone is enough and is simpler. We can
> build
> > > > > tools to
> > > > > > > > make
> > > > > > > > > > the
> > > > > > > > > > > configuration at different granularities easier. The
> > > > > definition of
> > > > > > > > > > > log.retention.ms is "The number of milliseconds to
> keep a
> > > log
> > > > > file
> > > > > > > > > > before
> > > > > > > > > > > deleting it". The deletion is independent of whether
> > > tiering is
> > > > > > > > enabled
> > > > > > > > > > or
> > > > > > > > > > > not. If this changes to just the local portion of the
> > > data, we
> > > > > are
> > > > > > > > > > changing
> > > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > > >
> > > > > > > > > > > We are fine with either way. We can go with
> > > log.retention.xxxx
> > > > > as the
> > > > > > > > > > > effective log retention instead of local log retention.
> > > With
> > > > > this
> > > > > > > > > > > convention, we need to introduce  local.log.retention
> > > instead
> > > > > of
> > > > > > > > > > > remote.log.retention.ms that we proposed. If
> > > log.retention.ms
> > > > > as -1
> > > > > > > > > > > then remote retention is also considered as unlimited
> but
> > > user
> > > > > should
> > > > > > > > > > > be able to set the local.retention.ms.
> > > > > > > > > > > So, we need to introduce local.log.retention.ms and
> > > > > > > > > > > local.log.retention.bytes which should  always  be <=
> > > > > > > > > > > log.retention.ms/bytes respectively.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <
> jun@confluent.io>
> > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the reply. A few quick comments below.
> > > > > > > > > > > >
> > > > > > > > > > > > 1001. I was thinking that you could just use the
> tiered
> > > > > metadata to
> > > > > > > > > do
> > > > > > > > > > > the
> > > > > > > > > > > > reconciliation. The tiered metadata contains offset
> > > ranges
> > > > > and
> > > > > > > > epoch
> > > > > > > > > > > > history. Those should be enough for reconciliation
> > > purposes.
> > > > > > > > > > > >
> > > > > > > > > > > > 1003.3 Having just a serverEndpoint string is
> probably
> > > not
> > > > > enough.
> > > > > > > > > > > > Connecting to a Kafka cluster may need various
> security
> > > > > > > > credentials.
> > > > > > > > > We
> > > > > > > > > > > can
> > > > > > > > > > > > make RLMM configurable and pass in the properties
> > > through the
> > > > > > > > > > configure()
> > > > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > > > >
> > > > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > > > > log.retention.minutes,
> > > > > > > > > > and
> > > > > > > > > > > > added log.retention.ms later. If we are adding a new
> > > > > > > > configuration,
> > > > > > > > > ms
> > > > > > > > > > > > level config alone is enough and is simpler. We can
> build
> > > > > tools to
> > > > > > > > > make
> > > > > > > > > > > the
> > > > > > > > > > > > configuration at different granularities easier. The
> > > > > definition of
> > > > > > > > > > > > log.retention.ms is "The number of milliseconds to
> keep
> > > a
> > > > > log file
> > > > > > > > > > > before
> > > > > > > > > > > > deleting it". The deletion is independent of whether
> > > tiering
> > > > > is
> > > > > > > > > enabled
> > > > > > > > > > > or
> > > > > > > > > > > > not. If this changes to just the local portion of the
> > > data,
> > > > > we are
> > > > > > > > > > > changing
> > > > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thank you for the comments! Ying, Harsha and I
> > > discussed
> > > > > and put
> > > > > > > > > our
> > > > > > > > > > > > > comments below.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1001. The KIP described a few scenarios of unclean
> > > leader
> > > > > > > > > elections.
> > > > > > > > > > > This
> > > > > > > > > > > > > is very useful, but I am wondering if this is the
> best
> > > > > approach.
> > > > > > > > My
> > > > > > > > > > > > > understanding of the proposed approach is to allow
> the
> > > new
> > > > > > > > > (unclean)
> > > > > > > > > > > leader
> > > > > > > > > > > > > to take new messages immediately. While this
> increases
> > > > > > > > > availability,
> > > > > > > > > > it
> > > > > > > > > > > > > creates the problem that there could be multiple
> > > > > conflicting
> > > > > > > > > segments
> > > > > > > > > > > in
> > > > > > > > > > > > > the remote store for the same offset range. This
> seems
> > > to
> > > > > make it
> > > > > > > > > > > harder
> > > > > > > > > > > > > for RLMM to determine which archived log segments
> > > contain
> > > > > the
> > > > > > > > > correct
> > > > > > > > > > > data.
> > > > > > > > > > > > > For example, an archived log segment could at one
> time
> > > be
> > > > > the
> > > > > > > > > correct
> > > > > > > > > > > data,
> > > > > > > > > > > > > but be changed to incorrect data after an unclean
> > > leader
> > > > > > > > election.
> > > > > > > > > An
> > > > > > > > > > > > > alternative approach is to let the unclean leader
> use
> > > the
> > > > > > > > archived
> > > > > > > > > > > data as
> > > > > > > > > > > > > the source of truth. So, when the new (unclean)
> leader
> > > > > takes
> > > > > > > > over,
> > > > > > > > > it
> > > > > > > > > > > first
> > > > > > > > > > > > > reconciles the local data based on the archived
> data
> > > before
> > > > > > > > taking
> > > > > > > > > > new
> > > > > > > > > > > > > messages. This makes the job of RLMM a bit easier
> > > since all
> > > > > > > > > archived
> > > > > > > > > > > data
> > > > > > > > > > > > > are considered correct. This increases
> availability a
> > > bit.
> > > > > > > > However,
> > > > > > > > > > > since
> > > > > > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Firstly, We don't want to assume the remote
> storage is
> > > more
> > > > > > > > > reliable
> > > > > > > > > > > than
> > > > > > > > > > > > > Kafka. Kafka unclean leader election usually
> happens
> > > when
> > > > > there
> > > > > > > > is
> > > > > > > > > a
> > > > > > > > > > > large
> > > > > > > > > > > > > scale outage that impacts multiple racks (or even
> > > multiple
> > > > > > > > > > availability
> > > > > > > > > > > > > zones). In such a case, the remote storage may be
> > > > > unavailable or
> > > > > > > > > > > unstable.
> > > > > > > > > > > > > Pulling a large amount of data from the remote
> storage
> > > to
> > > > > > > > reconcile
> > > > > > > > > > the
> > > > > > > > > > > > > local data may also exacerbate the outage. With the
> > > current
> > > > > > > > design,
> > > > > > > > > > > the new
> > > > > > > > > > > > > leader can start working even when the remote
> storage
> > > is
> > > > > > > > > temporarily
> > > > > > > > > > > > > unavailable.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Secondly, it is not easier to implement the
> reconciling
> > > > > logic at
> > > > > > > > > the
> > > > > > > > > > > leader
> > > > > > > > > > > > > side. It can take a long time for the new leader to
> > > > > download the
> > > > > > > > > > remote
> > > > > > > > > > > > > data and rebuild local producer id / leader epoch
> > > > > information.
> > > > > > > > > During
> > > > > > > > > > > this
> > > > > > > > > > > > > period, the leader cannot accept any requests from
> the
> > > > > clients
> > > > > > > > and
> > > > > > > > > > > > > followers. We have to introduce a new state for the
> > > > > leader, and a
> > > > > > > > > new
> > > > > > > > > > > error
> > > > > > > > > > > > > code to let the clients / followers know what is
> > > happening.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > > > > RemoteStorageManager.
> > > > > > > > > > > We
> > > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For
> all
> > > other
> > > > > > > > methods,
> > > > > > > > > > we
> > > > > > > > > > > pass
> > > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Nice catch, we can have the
> RemoteLogSegmentMetadata
> > > for
> > > > > > > > > > copyLogSegment
> > > > > > > > > > > > > too.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > > inclusive
> > > > > or
> > > > > > > > > > exclusive?
> > > > > > > > > > > > >
> > > > > > > > > > > > > It is inclusive.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > > leaderEpoch
> > > > > > > > history
> > > > > > > > > > for
> > > > > > > > > > > a
> > > > > > > > > > > > > partition.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Yes, updated the KIP with the new method.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1002.4 Could you define the type of
> > > > > RemoteLogSegmentContext?
> > > > > > > > > > > > >
> > > > > > > > > > > > > This is removed in the latest code and it is not
> > > needed.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1003.1 I am not sure why we need both of the
> following
> > > > > methods
> > > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine them
> into
> > > > > one that
> > > > > > > > > > takes
> > > > > > > > > > > in
> > > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > > >     RemoteLogSegmentId
> > > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > >
> > > > > > > > > > > > > Good point, these can be merged for now. I guess we
> > > needed
> > > > > them
> > > > > > > > in
> > > > > > > > > > > earlier
> > > > > > > > > > > > > version of the implementation but it is not needed
> now.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1003.2 There seems to be some inconsistencies in
> the
> > > > > methods
> > > > > > > > > below. I
> > > > > > > > > > > am
> > > > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata
> and the
> > > > > other
> > > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > > >     void
> > > putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > > >     void
> > > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > >
> > > > > > > > > > > > > RLMM stores RemoteLogSegmentMetadata which is
> > > identified by
> > > > > > > > > > > > > RemoteLogsSegmentId. So, when it is added it takes
> > > > > > > > > > > > > RemoteLogSegmentMetadata. `delete` operation needs
> only
> > > > > > > > > > > RemoteLogsSegmentId
> > > > > > > > > > > > > as RemoteLogSegmentMetadata can be identified with
> > > > > > > > > > RemoteLogsSegmentId.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > > > serverEndpoint), what
> > > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > > >
> > > > > > > > > > > > > This can be used by RLMM implementation to connect
> to
> > > the
> > > > > local
> > > > > > > > > Kafka
> > > > > > > > > > > > > cluster. Incase of default  implementation, it is
> used
> > > in
> > > > > > > > > > initializing
> > > > > > > > > > > > > kafka clients connecting to the local cluster.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1004. It would be useful to document how all the
> new
> > > APIs
> > > > > are
> > > > > > > > being
> > > > > > > > > > > used.
> > > > > > > > > > > > > For example, when is
> > > > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > > being
> > > > > > > > > > > set
> > > > > > > > > > > > > and used? How are
> > > > > > > > > > > > >
> > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > > > > > > > > used?
> > > > > > > > > > > > >
> > > > > > > > > > > > > RLMM APIs are going through the changes and they
> > > should be
> > > > > ready
> > > > > > > > > in a
> > > > > > > > > > > few
> > > > > > > > > > > > > days. I will update the KIP and the mail  thread
> once
> > > they
> > > > > are
> > > > > > > > > ready.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1005. Handling partition deletion: The KIP says
> "RLMM
> > > will
> > > > > > > > > eventually
> > > > > > > > > > > > > delete these segments by using
> RemoteStorageManager."
> > > Which
> > > > > > > > replica
> > > > > > > > > > > does
> > > > > > > > > > > > > this logic?
> > > > > > > > > > > > >
> > > > > > > > > > > > > This is a good point. When a topic is deleted, it
> will
> > > not
> > > > > have
> > > > > > > > any
> > > > > > > > > > > > > leader/followers to do the cleanup. We will have a
> > > cleaner
> > > > > agent
> > > > > > > > > on a
> > > > > > > > > > > > > single broker in the cluster to do this cleanup, we
> > > plan
> > > > > to add
> > > > > > > > > that
> > > > > > > > > > in
> > > > > > > > > > > > > controller broker.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1006. "If there are any failures in removing
> remote log
> > > > > segments
> > > > > > > > > then
> > > > > > > > > > > those
> > > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > > remote-log-segment-id)
> > > > > > > > > > > from
> > > > > > > > > > > > > that topic and clean them up from remote storage.
> "
> > > Not
> > > > > sure if
> > > > > > > > > it's
> > > > > > > > > > > worth
> > > > > > > > > > > > > the complexity of adding another topic. Could we
> just
> > > > > retry?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Sure, we can keep this simpler for now by logging
> an
> > > error
> > > > > after
> > > > > > > > > > > retries.
> > > > > > > > > > > > > We can give users a better way to process this in
> > > future.
> > > > > Oneway
> > > > > > > > > can
> > > > > > > > > > > be a
> > > > > > > > > > > > > dead letter topic which can be configured by the
> user.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> > > > > existing
> > > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > > >
> > > > > > > > > > > > > We have 2 types of delayed operations waiting for 2
> > > > > different
> > > > > > > > > events.
> > > > > > > > > > > > > DelayedFetch waits for new messages from producers.
> > > > > > > > > > DelayedRemoteFetch
> > > > > > > > > > > > > waits for the remote-storage-read-task to finish.
> When
> > > > > either of
> > > > > > > > > the
> > > > > > > > > > 2
> > > > > > > > > > > > > events happens, we only want to notify one type of
> the
> > > > > delayed
> > > > > > > > > > > operations.
> > > > > > > > > > > > > It would be inefficient to put 2 types of delayed
> > > > > operations in
> > > > > > > > one
> > > > > > > > > > > > > purgatory, as the tryComplete() methods of the
> delayed
> > > > > operations
> > > > > > > > > can
> > > > > > > > > > > be
> > > > > > > > > > > > > triggered by irrelevant events.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > > > remote.log.retention.minutes,
> > > > > > > > > > > > > remote.log.retention.hours: It seems that we just
> need
> > > the
> > > > > ms
> > > > > > > > one.
> > > > > > > > > > > Also,
> > > > > > > > > > > > > are we changing the meaning of existing config
> > > > > log.retention.ms
> > > > > > > > to
> > > > > > > > > > > mean
> > > > > > > > > > > > > the
> > > > > > > > > > > > > local retention? For backward compatibility, it's
> > > better
> > > > > to not
> > > > > > > > > > change
> > > > > > > > > > > the
> > > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > > >
> > > > > > > > > > > > > We agree that we only need remote.log.retention.ms
> .
> > > But,
> > > > > the
> > > > > > > > > > existing
> > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > configuration
> > > > > > > > > > > > > has 3 properties (log.retention.ms,
> > > log.retention.minutes,
> > > > > > > > > > > > > log.retention.hours). We just
> > > > > > > > > > > > > want to keep consistent with the existing
> properties.
> > > > > > > > > > > > > Existing log.retention.xxxx config is about log
> > > retention
> > > > > in
> > > > > > > > > broker’s
> > > > > > > > > > > > > storage which is local. It should be easy for
> users to
> > > > > configure
> > > > > > > > > > > partition
> > > > > > > > > > > > > storage with local retention and remote retention
> > > based on
> > > > > their
> > > > > > > > > > usage.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the
> topic
> > > > > level?
> > > > > > > > > > > > >
> > > > > > > > > > > > > We can introduce topic level config for the same
> > > remote.log
> > > > > > > > > settings.
> > > > > > > > > > > User
> > > > > > > > > > > > > can set the desired config while creating the
> topic.
> > > > > > > > > > > > > remote.log.storage.enable property is not allowed
> to be
> > > > > updated
> > > > > > > > > after
> > > > > > > > > > > the
> > > > > > > > > > > > > topic is created. Other remote.log.* properties
> can be
> > > > > modified.
> > > > > > > > We
> > > > > > > > > > > will
> > > > > > > > > > > > > support flipping remote.log.storage.enable in next
> > > > > versions.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1009. It would be useful to list all limitations
> in a
> > > > > separate
> > > > > > > > > > section:
> > > > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a
> topic
> > > from
> > > > > delete
> > > > > > > > > to
> > > > > > > > > > > > > compact and vice versa allowed when tiering is
> enabled?
> > > > > > > > > > > > >
> > > > > > > > > > > > > +1 to have limitations in a separate section. We
> will
> > > > > update the
> > > > > > > > > KIP
> > > > > > > > > > > with
> > > > > > > > > > > > > that.
> > > > > > > > > > > > > Topic  created with effective value for
> > > remote.log.enabled
> > > > > as
> > > > > > > > true,
> > > > > > > > > > > can not
> > > > > > > > > > > > > change its retention policy from delete to compact.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1010. Thanks for performance numbers. Are those
> with
> > > > > RocksDB as
> > > > > > > > the
> > > > > > > > > > > cache?
> > > > > > > > > > > > >
> > > > > > > > > > > > > No, We have not yet added RocksDB support. This is
> > > based on
> > > > > > > > > in-memory
> > > > > > > > > > > map
> > > > > > > > > > > > > representation. We will add that support and update
> > > this
> > > > > thread
> > > > > > > > > after
> > > > > > > > > > > > > updating the KIP with the numbers.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Satish.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <
> > > jun@confluent.io>
> > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > > below.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1000. Regarding Colin's question on querying the
> > > metadata
> > > > > > > > > directly
> > > > > > > > > > > in the
> > > > > > > > > > > > > > remote block store. One issue is that not all
> block
> > > > > stores
> > > > > > > > offer
> > > > > > > > > > the
> > > > > > > > > > > > > needed
> > > > > > > > > > > > > > api to query the metadata. For example, S3 only
> > > offers
> > > > > an api
> > > > > > > > to
> > > > > > > > > > list
> > > > > > > > > > > > > > objects under a prefix and this api has the
> eventual
> > > > > > > > consistency
> > > > > > > > > > > > > semantic.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1001. The KIP described a few scenarios of
> unclean
> > > leader
> > > > > > > > > > elections.
> > > > > > > > > > > This
> > > > > > > > > > > > > > is very useful, but I am wondering if this is the
> > > best
> > > > > > > > approach.
> > > > > > > > > My
> > > > > > > > > > > > > > understanding of the proposed approach is to
> allow
> > > the
> > > > > new
> > > > > > > > > > (unclean)
> > > > > > > > > > > > > leader
> > > > > > > > > > > > > > to take new messages immediately. While this
> > > increases
> > > > > > > > > > availability,
> > > > > > > > > > > it
> > > > > > > > > > > > > > creates the problem that there could be multiple
> > > > > conflicting
> > > > > > > > > > > segments in
> > > > > > > > > > > > > > the remote store for the same offset range. This
> > > seems
> > > > > to make
> > > > > > > > it
> > > > > > > > > > > harder
> > > > > > > > > > > > > > for RLMM to determine which archived log segments
> > > > > contain the
> > > > > > > > > > correct
> > > > > > > > > > > > > data.
> > > > > > > > > > > > > > For example, an archived log segment could at one
> > > time
> > > > > be the
> > > > > > > > > > correct
> > > > > > > > > > > > > data,
> > > > > > > > > > > > > > but be changed to incorrect data after an unclean
> > > leader
> > > > > > > > > election.
> > > > > > > > > > An
> > > > > > > > > > > > > > alternative approach is to let the unclean leader
> > > use the
> > > > > > > > > archived
> > > > > > > > > > > data
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > the source of truth. So, when the new (unclean)
> > > leader
> > > > > takes
> > > > > > > > > over,
> > > > > > > > > > it
> > > > > > > > > > > > > first
> > > > > > > > > > > > > > reconciles the local data based on the archived
> data
> > > > > before
> > > > > > > > > taking
> > > > > > > > > > > new
> > > > > > > > > > > > > > messages. This makes the job of RLMM a bit easier
> > > since
> > > > > all
> > > > > > > > > > archived
> > > > > > > > > > > data
> > > > > > > > > > > > > > are considered correct. This increases
> availability a
> > > > > bit.
> > > > > > > > > However,
> > > > > > > > > > > since
> > > > > > > > > > > > > > unclean leader elections are rare, this may be
> ok.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > > > > > RemoteStorageManager. We
> > > > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For
> all
> > > > > other
> > > > > > > > > methods,
> > > > > > > > > > > we
> > > > > > > > > > > > > pass
> > > > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > > > > inclusive or
> > > > > > > > > > > exclusive?
> > > > > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > > > > leaderEpoch
> > > > > > > > > history
> > > > > > > > > > > for a
> > > > > > > > > > > > > > partition.
> > > > > > > > > > > > > > 1002.4 Could you define the type of
> > > > > RemoteLogSegmentContext?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > > > > 1003.1 I am not sure why we need both of the
> > > following
> > > > > methods
> > > > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine
> them
> > > into
> > > > > one
> > > > > > > > that
> > > > > > > > > > > takes in
> > > > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > > > >     RemoteLogSegmentId
> > > > > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > > 1003.2 There seems to be some inconsistencies in
> the
> > > > > methods
> > > > > > > > > below.
> > > > > > > > > > > I am
> > > > > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata
> and
> > > the
> > > > > other
> > > > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > > > >     void
> > > putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > > > >     void
> > > > > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > > > serverEndpoint),
> > > > > > > > what
> > > > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1004. It would be useful to document how all the
> new
> > > > > APIs are
> > > > > > > > > being
> > > > > > > > > > > used.
> > > > > > > > > > > > > > For example, when is
> > > > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > > > > being set
> > > > > > > > > > > > > > and used? How are
> > > > > > > > > > > > > >
> > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > > > > > > > being
> > > > > > > > > > > used?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1005. Handling partition deletion: The KIP says
> "RLMM
> > > > > will
> > > > > > > > > > eventually
> > > > > > > > > > > > > > delete these segments by using
> RemoteStorageManager."
> > > > > Which
> > > > > > > > > replica
> > > > > > > > > > > does
> > > > > > > > > > > > > > this logic?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1006. "If there are any failures in removing
> remote
> > > log
> > > > > > > > segments
> > > > > > > > > > then
> > > > > > > > > > > > > those
> > > > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > > > remote-log-segment-id)
> > > > > > > > > > > from
> > > > > > > > > > > > > > that topic and clean them up from remote
> storage.  "
> > > Not
> > > > > sure
> > > > > > > > if
> > > > > > > > > > it's
> > > > > > > > > > > > > worth
> > > > > > > > > > > > > > the complexity of adding another topic. Could we
> just
> > > > > retry?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse
> the
> > > > > existing
> > > > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > > > remote.log.retention.minutes,
> > > > > > > > > > > > > > remote.log.retention.hours: It seems that we just
> > > need
> > > > > the ms
> > > > > > > > > one.
> > > > > > > > > > > Also,
> > > > > > > > > > > > > > are we changing the meaning of existing config
> > > > > > > > log.retention.ms
> > > > > > > > > to
> > > > > > > > > > > mean
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > local retention? For backward compatibility, it's
> > > better
> > > > > to not
> > > > > > > > > > > change
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the
> > > topic
> > > > > level?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1009. It would be useful to list all limitations
> in a
> > > > > separate
> > > > > > > > > > > section:
> > > > > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a
> topic
> > > > > from
> > > > > > > > delete
> > > > > > > > > > to
> > > > > > > > > > > > > > compact and vice versa allowed when tiering is
> > > enabled?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1010. Thanks for performance numbers. Are those
> with
> > > > > RocksDB as
> > > > > > > > > the
> > > > > > > > > > > > > cache?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <
> > > > > harsha.ch@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > >                Thats not what we said in the
> > > previous
> > > > > email.
> > > > > > > > > RLMM
> > > > > > > > > > > is
> > > > > > > > > > > > > > > pluggable storage and by running numbers even
> 1PB
> > > data
> > > > > you do
> > > > > > > > > not
> > > > > > > > > > > need
> > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > than 10GB local storage.
> > > > > > > > > > > > > > > If in future this becomes a blocker for any
> users
> > > we
> > > > > can
> > > > > > > > > revisit
> > > > > > > > > > > but
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > does not warrant another implementation at this
> > > point
> > > > > to push
> > > > > > > > > the
> > > > > > > > > > > data
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > We can ofcourse implement another RLMM that is
> > > > > optional for
> > > > > > > > > users
> > > > > > > > > > > to
> > > > > > > > > > > > > > > configure to push to remote. But that doesn't
> need
> > > to
> > > > > be
> > > > > > > > > > addressed
> > > > > > > > > > > in
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Harsha
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe <
> > > > > > > > > cmccabe@apache.org
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Ying,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for the response.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > It sounds like you agree that storing the
> > > metadata
> > > > > in the
> > > > > > > > > > remote
> > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > would be a better design overall.  Given that
> > > that's
> > > > > true,
> > > > > > > > is
> > > > > > > > > > > there
> > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > reason to include the worse implementation
> based
> > > on
> > > > > > > > RocksDB?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Choosing a long-term metadata store is not
> > > something
> > > > > that
> > > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > lightly.  It can take users years to migrate
> from
> > > > > metadata
> > > > > > > > > > store
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > other.  I also don't think it's realistic or
> > > > > desirable for
> > > > > > > > > > users
> > > > > > > > > > > to
> > > > > > > > > > > > > > write
> > > > > > > > > > > > > > > > their own metadata stores.  Even assuming
> that
> > > they
> > > > > could
> > > > > > > > do
> > > > > > > > > a
> > > > > > > > > > > good
> > > > > > > > > > > > > job
> > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > this, it would create huge fragmentation in
> the
> > > Kafka
> > > > > > > > > > ecosystem.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng
> wrote:
> > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Satish and I are still discussing some
> details
> > > > > about how
> > > > > > > > to
> > > > > > > > > > > handle
> > > > > > > > > > > > > > > > > transactions / producer ids. Satish is
> going to
> > > > > make some
> > > > > > > > > > minor
> > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > RLMM API and other parts. Other than that,
> we
> > > have
> > > > > > > > finished
> > > > > > > > > > > > > updating
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I agree with Colin that the current design
> of
> > > using
> > > > > > > > rocksDB
> > > > > > > > > > is
> > > > > > > > > > > not
> > > > > > > > > > > > > > > > > optimal. But this design is simple and
> should
> > > work
> > > > > for
> > > > > > > > > almost
> > > > > > > > > > > all
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > existing Kafka users. RLMM is a plugin.
> Users
> > > can
> > > > > replace
> > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > their own RLMM implementation, if needed.
> So, I
> > > > > think we
> > > > > > > > > can
> > > > > > > > > > > keep
> > > > > > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > > > > for now. What do you think?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Ying
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> > > > > > > > jun@confluent.io>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks for the update. It's good to see
> the
> > > > > progress on
> > > > > > > > > > this.
> > > > > > > > > > > > > > Please
> > > > > > > > > > > > > > > > let us
> > > > > > > > > > > > > > > > > > know when you are done updating the KIP
> wiki.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying
> Zheng
> > > > > > > > > > > > > <yingz@uber.com.invalid
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Satish and I have added more design
> > > details in
> > > > > the
> > > > > > > > KIP,
> > > > > > > > > > > > > including
> > > > > > > > > > > > > > > > how to
> > > > > > > > > > > > > > > > > > > keep consistency between replicas
> > > (especially
> > > > > when
> > > > > > > > > there
> > > > > > > > > > is
> > > > > > > > > > > > > > > > leadership
> > > > > > > > > > > > > > > > > > > changes / log truncations) and new
> > > metrics. We
> > > > > also
> > > > > > > > > made
> > > > > > > > > > > some
> > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > minor
> > > > > > > > > > > > > > > > > > > changes in the doc. We will finish the
> KIP
> > > > > changes in
> > > > > > > > > the
> > > > > > > > > > > next
> > > > > > > > > > > > > > > > couple of
> > > > > > > > > > > > > > > > > > > days. We will let you know when we are
> > > done.
> > > > > Most of
> > > > > > > > > the
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > already updated to the wiki KIP. You
> can
> > > take
> > > > > a look.
> > > > > > > > > But
> > > > > > > > > > > it's
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > As for the implementation, the code is
> > > mostly
> > > > > done
> > > > > > > > and
> > > > > > > > > we
> > > > > > > > > > > > > already
> > > > > > > > > > > > > > > had
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > feature tests / system tests. I have
> added
> > > the
> > > > > > > > > > performance
> > > > > > > > > > > test
> > > > > > > > > > > > > > > > results
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > the KIP. However the recent design
> changes
> > > > > (e.g.
> > > > > > > > leader
> > > > > > > > > > > epoch
> > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > management / log truncation / some of
> the
> > > new
> > > > > > > > metrics)
> > > > > > > > > > > have not
> > > > > > > > > > > > > > > been
> > > > > > > > > > > > > > > > > > > implemented yet. It will take about 2
> weeks
> > > > > for us to
> > > > > > > > > > > implement
> > > > > > > > > > > > > > > > after you
> > > > > > > > > > > > > > > > > > > review and agree with those design
> changes.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun Rao
> <
> > > > > > > > > jun@confluent.io
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Any new updates on the KIP? This
> feature
> > > is
> > > > > one of
> > > > > > > > > the
> > > > > > > > > > > most
> > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > most requested features in Apache
> Kafka
> > > > > right now.
> > > > > > > > It
> > > > > > > > > > > would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > helpful
> > > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > we can make sustained progress on
> this.
> > > > > Could you
> > > > > > > > > share
> > > > > > > > > > > how
> > > > > > > > > > > > > far
> > > > > > > > > > > > > > > > along
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > the design/implementation right now?
> Is
> > > there
> > > > > > > > > anything
> > > > > > > > > > > that
> > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > As for "transactional support" and
> > > "follower
> > > > > > > > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > > > > > > > further comments from me as long as
> the
> > > > > producer
> > > > > > > > > state
> > > > > > > > > > > and
> > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > be restored properly from the object
> > > store
> > > > > when
> > > > > > > > > needed.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM Satish
> > > > > Duggana <
> > > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > We did not want to add many
> > > implementation
> > > > > > > > details
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > But we
> > > > > > > > > > > > > > > > > > > > > decided to add them in the KIP as
> > > appendix
> > > > > or
> > > > > > > > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > > > > > > > follower fetch protocol) to
> describe
> > > the
> > > > > flow
> > > > > > > > with
> > > > > > > > > > the
> > > > > > > > > > > main
> > > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > > > That will answer most of the
> queries. I
> > > > > will
> > > > > > > > update
> > > > > > > > > > on
> > > > > > > > > > > this
> > > > > > > > > > > > > > > mail
> > > > > > > > > > > > > > > > > > > > > thread when the respective
> sections are
> > > > > updated.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM
> > > Alexandre
> > > > > Dupriez
> > > > > > > > > > > > > > > > > > > > > <al...@gmail.com>
> wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > A couple of questions specific
> to the
> > > > > section
> > > > > > > > > > > "Follower
> > > > > > > > > > > > > > > > > > > > > > Requests/Replication", pages
> 16:17
> > > in the
> > > > > > > > design
> > > > > > > > > > > document
> > > > > > > > > > > > > > > [1].
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900. It is mentioned that
> followers
> > > fetch
> > > > > > > > > auxiliary
> > > > > > > > > > > > > states
> > > > > > > > > > > > > > > > from the
> > > > > > > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900.a Does the consistency model
> of
> > > the
> > > > > > > > external
> > > > > > > > > > > storage
> > > > > > > > > > > > > > > > impacts
> > > > > > > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > > > > > > > of leader epochs and other
> auxiliary
> > > > > data?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900.b What are the benefits of
> using
> > > a
> > > > > > > > mechanism
> > > > > > > > > to
> > > > > > > > > > > store
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > > > > > > the leader epochs which is
> different
> > > > > from other
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > tiered segments? What are the
> > > benefits of
> > > > > > > > > > retrieving
> > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > on-demand from the follower
> rather
> > > than
> > > > > relying
> > > > > > > > > on
> > > > > > > > > > > > > > > propagation
> > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > topic __remote_log_metadata? What
> > > are the
> > > > > > > > > > advantages
> > > > > > > > > > > over
> > > > > > > > > > > > > > > > using a
> > > > > > > > > > > > > > > > > > > > > > dedicated control structure
> (e.g. a
> > > new
> > > > > record
> > > > > > > > > > type)
> > > > > > > > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > > > > > > > this topic? Since in the
> document,
> > > > > different
> > > > > > > > > > control
> > > > > > > > > > > > > paths
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > operating in the system, how are
> the
> > > > > metadata
> > > > > > > > > > stored
> > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > __remote_log_metadata [which also
> > > > > include the
> > > > > > > > > epoch
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > which offloaded a segment] and
> the
> > > remote
> > > > > > > > > auxiliary
> > > > > > > > > > > > > states,
> > > > > > > > > > > > > > > > kept in
> > > > > > > > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900.c A follower can encounter an
> > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > > > > > > > this in response to a Fetch or
> > > > > > > > > OffsetForLeaderEpoch
> > > > > > > > > > > > > > request?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900.d What happens if, after a
> > > follower
> > > > > > > > > encountered
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE
> > > response,
> > > > > its
> > > > > > > > > > > attempts to
> > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > leader epochs fail (for instance,
> > > > > because the
> > > > > > > > > > remote
> > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > temporarily unavailable)? Does
> the
> > > > > follower
> > > > > > > > > > > fallbacks to
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > mode
> > > > > > > > > > > > > > > > > > where
> > > > > > > > > > > > > > > > > > > > > > it ignores tiered segments, and
> > > applies
> > > > > > > > > truncation
> > > > > > > > > > > using
> > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > > > > available information? What
> happens
> > > when
> > > > > access
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > > > is restored? How is the replica
> > > lineage
> > > > > > > > inferred
> > > > > > > > > by
> > > > > > > > > > > the
> > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > epochs reconciled with the
> follower's
> > > > > replica
> > > > > > > > > > > lineage,
> > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > evolved? Does the follower
> remember
> > > > > fetching
> > > > > > > > > > > auxiliary
> > > > > > > > > > > > > > states
> > > > > > > > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > > > > > > > in the past and attempt
> > > reconciliation?
> > > > > Is
> > > > > > > > there
> > > > > > > > > a
> > > > > > > > > > > plan
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > offer
> > > > > > > > > > > > > > > > > > > > > > different strategies in this
> > > scenario,
> > > > > > > > > configurable
> > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900.e Is the leader epoch cache
> > > > > offloaded with
> > > > > > > > > > every
> > > > > > > > > > > > > > segment?
> > > > > > > > > > > > > > > > Or
> > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > > a new checkpoint is detected? If
> that
> > > > > > > > information
> > > > > > > > > > is
> > > > > > > > > > > not
> > > > > > > > > > > > > > > always
> > > > > > > > > > > > > > > > > > > > > > offloaded to avoid duplicating
> data,
> > > how
> > > > > does
> > > > > > > > the
> > > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > > > satisfy the request to retrieve
> it?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900.f Since the leader epoch
> cache
> > > > > covers the
> > > > > > > > > > entire
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > > > > > > > what happens if, after a leader
> epoch
> > > > > cache
> > > > > > > > file
> > > > > > > > > is
> > > > > > > > > > > > > > offloaded
> > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > given segment, the local epoch
> cache
> > > is
> > > > > > > > truncated
> > > > > > > > > > > [not
> > > > > > > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > a range of offset included in
> tiered
> > > > > segments]?
> > > > > > > > > How
> > > > > > > > > > > are
> > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > local leader epoch caches kept
> > > > > consistent?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900.g Consumer can also use
> leader
> > > > > epochs (e.g.
> > > > > > > > > to
> > > > > > > > > > > enable
> > > > > > > > > > > > > > > > fencing
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > protect against stale leaders).
> What
> > > > > > > > differences
> > > > > > > > > > > would
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > between consumer and follower
> > > fetches?
> > > > > > > > > Especially,
> > > > > > > > > > > would
> > > > > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > > > > > > also fetch leader epoch
> information
> > > from
> > > > > the
> > > > > > > > > remote
> > > > > > > > > > > > > > storage?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 900.h Assume a newly elected
> leader
> > > of a
> > > > > > > > > > > topic-partition
> > > > > > > > > > > > > > > > detects
> > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > recent segments are available in
> the
> > > > > external
> > > > > > > > > > > storage,
> > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > epochs
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > its local epoch. Does it ignore
> these
> > > > > segments
> > > > > > > > > and
> > > > > > > > > > > their
> > > > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > > > > epoch-to-offset vectors? Or try
> to
> > > > > reconstruct
> > > > > > > > > its
> > > > > > > > > > > local
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > lineage based on the data
> remotely
> > > > > available?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > >
> > >
> https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55,
> Satish
> > > > > Duggana <
> > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > > > > > Please let us know if you have
> any
> > > > > comments
> > > > > > > > on
> > > > > > > > > > > > > > > "transactional
> > > > > > > > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > > > > > > > and "follower
> requests/replication"
> > > > > mentioned
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > wiki.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25 PM
> > > Satish
> > > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks Jun for your comments.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >100. It would be useful to
> > > provide
> > > > > more
> > > > > > > > > > details
> > > > > > > > > > > on
> > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > > > > are used. Otherwise, it's kind of
> hard
> > > to
> > > > > really
> > > > > > > > > > assess
> > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > > > > > apis are sufficient/redundant. A
> few
> > > > > examples
> > > > > > > > > below.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > We will update the wiki and
> let
> > > you
> > > > > know.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >100.1 deleteRecords seems to
> > > only
> > > > > advance
> > > > > > > > > the
> > > > > > > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > Log. How does that trigger the
> > > deletion of
> > > > > remote
> > > > > > > > > log
> > > > > > > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > RLMTask for leader partition
> > > > > periodically
> > > > > > > > > > checks
> > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > remote log segments earlier
> to
> > > > > > > > logStartOffset
> > > > > > > > > > > and the
> > > > > > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > > > > > > remote log segment metadata
> and
> > > data
> > > > > are
> > > > > > > > > > deleted
> > > > > > > > > > > by
> > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > > RLMM
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >100.2 stopReplica with
> deletion
> > > is
> > > > > used
> > > > > > > > in 2
> > > > > > > > > > > cases
> > > > > > > > > > > > > (a)
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > reassignment; (b) topic deletion.
> We
> > > only
> > > > > want to
> > > > > > > > > > > delete
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > > > > metadata in the second case. Also,
> in
> > > the
> > > > > second
> > > > > > > > > > case,
> > > > > > > > > > > who
> > > > > > > > >
> > >
>
>

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

Posted by Satish Duggana <sa...@gmail.com>.
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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
https://cwiki.apache.org/confluence/display/KAFKA/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
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' 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 <kp...@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
> 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' 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 <sa...@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 <ju...@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 <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > >
> > > > Satish.
> > > >
> > > >
> > > > On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana <
> > satish.duggana@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > > >
> > > > >
> > > > > Satish.
> > > > >
> > > > >
> > > > > On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng <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 <ju...@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
> > <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
> > > > > > > > unclean-leader-election.
> > > > > > > > 2) Since the user set local retention time (or local retention
> > > > bytes), I
> > > > > > > > think we are expected to
> > > > > > > > keep that much local data when possible (avoid truncating all
> > the
> > > > local
> > > > > > > > data). But, as you said,
> > > > > > > > unclean leader elections are very rare, this may not be a big
> > > > problem.
> > > > > > > >
> > > > > > > > The current design uses the leader broker as source-of-truth.
> > This
> > > > is
> > > > > > > > consistent with the
> > > > > > > > existing Kafka behavior.
> > > > > > > >
> > > > > > > > By using remote storage as the source-of-truth, the follower
> > logic
> > > > can
> > > > > > > be a
> > > > > > > > little simpler,
> > > > > > > > but the leader logic is going to be more complex. Overall, I
> > don't
> > > > see
> > > > > > > > there many benefits
> > > > > > > > of using remote storage as the source-of-truth.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > >
> > > > > > > > > Hi, Satish,
> > > > > > > > >
> > > > > > > > > Thanks for the reply.
> > > > > > > > >
> > > > > > > > > 1001. In your example, I was thinking that you could just
> > > > download the
> > > > > > > > > latest leader epoch from the object store. After that you
> > know
> > > > the
> > > > > > > leader
> > > > > > > > > should end with offset 1100. The leader will delete all its
> > > > local data
> > > > > > > > > before offset 1000 and start accepting new messages at offset
> > > > 1100.
> > > > > > > > > Consumer requests for messages before offset 1100 will be
> > served
> > > > from
> > > > > > > the
> > > > > > > > > object store. The benefit with this approach is that it's
> > > > simpler to
> > > > > > > > reason
> > > > > > > > > about who is the source of truth. The downside is slightly
> > > > increased
> > > > > > > > > unavailability window during unclean leader election. Since
> > > > unclean
> > > > > > > > leader
> > > > > > > > > elections are rare, I am not sure if this is a big concern.
> > > > > > > > >
> > > > > > > > > 1008. Yes, I think introducing sth like local.retention.ms
> > > > seems more
> > > > > > > > > consistent.
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > > > > > > satish.duggana@gmail.com
> > > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > HI Jun,
> > > > > > > > > > Thanks for your comments. We put our inline replies below.
> > > > > > > > > >
> > > > > > > > > > 1001. I was thinking that you could just use the tiered
> > > > metadata to
> > > > > > > do
> > > > > > > > > the
> > > > > > > > > > reconciliation. The tiered metadata contains offset ranges
> > and
> > > > epoch
> > > > > > > > > > history. Those should be enough for reconciliation
> > purposes.
> > > > > > > > > >
> > > > > > > > > > If we use remote storage as the source-of-truth during
> > > > > > > > > > unclean-leader-election, it's possible that after
> > > > reconciliation the
> > > > > > > > > > remote storage will have more recent data than the new
> > > > leader's local
> > > > > > > > > > storage. For example, the new leader's latest message is
> > > > offset 1000,
> > > > > > > > > > while the remote storage has message 1100. In such a case,
> > the
> > > > new
> > > > > > > > > > leader will have to download the messages from 1001 to
> > 1100,
> > > > before
> > > > > > > > > > accepting new messages from producers. Otherwise, there
> > would
> > > > be a
> > > > > > > gap
> > > > > > > > > > in the local data between 1000 and 1101.
> > > > > > > > > >
> > > > > > > > > > Moreover, with the current design, leader epoch history is
> > > > stored in
> > > > > > > > > > remote storage, rather than the metadata topic. We did
> > consider
> > > > > > > saving
> > > > > > > > > > epoch history in remote segment metadata. But the concern
> > is
> > > > that
> > > > > > > > > > there is currently no limit for the epoch history size.
> > > > > > > Theoretically,
> > > > > > > > > > if a user has a very long remote retention time and there
> > are
> > > > very
> > > > > > > > > > frequent leadership changes, the leader epoch history can
> > > > become too
> > > > > > > > > > long to fit into a regular Kafka message.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 1003.3 Having just a serverEndpoint string is probably not
> > > > enough.
> > > > > > > > > > Connecting to a Kafka cluster may need various security
> > > > credentials.
> > > > > > > We
> > > > > > > > > can
> > > > > > > > > > make RLMM configurable and pass in the properties through
> > the
> > > > > > > > configure()
> > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > >
> > > > > > > > > > RLMM and  RSM are already configurable and they take
> > > > properties which
> > > > > > > > > > start with "remote.log.metadata." and "remote.log.storage."
> > > > > > > > > > respectively and a few others. We have listener-name as the
> > > > config
> > > > > > > for
> > > > > > > > > > RLMM and other properties(like security) can be sent as you
> > > > > > > suggested.
> > > > > > > > > > We will update the KIP with the details.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > log.retention.minutes,
> > > > > > > > and
> > > > > > > > > > added log.retention.ms later. If we are adding a new
> > > > configuration,
> > > > > > > ms
> > > > > > > > > > level config alone is enough and is simpler. We can build
> > > > tools to
> > > > > > > make
> > > > > > > > > the
> > > > > > > > > > configuration at different granularities easier. The
> > > > definition of
> > > > > > > > > > log.retention.ms is "The number of milliseconds to keep a
> > log
> > > > file
> > > > > > > > > before
> > > > > > > > > > deleting it". The deletion is independent of whether
> > tiering is
> > > > > > > enabled
> > > > > > > > > or
> > > > > > > > > > not. If this changes to just the local portion of the
> > data, we
> > > > are
> > > > > > > > > changing
> > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > >
> > > > > > > > > > We are fine with either way. We can go with
> > log.retention.xxxx
> > > > as the
> > > > > > > > > > effective log retention instead of local log retention.
> > With
> > > > this
> > > > > > > > > > convention, we need to introduce  local.log.retention
> > instead
> > > > of
> > > > > > > > > > remote.log.retention.ms that we proposed. If
> > log.retention.ms
> > > > as -1
> > > > > > > > > > then remote retention is also considered as unlimited but
> > user
> > > > should
> > > > > > > > > > be able to set the local.retention.ms.
> > > > > > > > > > So, we need to introduce local.log.retention.ms and
> > > > > > > > > > local.log.retention.bytes which should  always  be <=
> > > > > > > > > > log.retention.ms/bytes respectively.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > Hi, Satish,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the reply. A few quick comments below.
> > > > > > > > > > >
> > > > > > > > > > > 1001. I was thinking that you could just use the tiered
> > > > metadata to
> > > > > > > > do
> > > > > > > > > > the
> > > > > > > > > > > reconciliation. The tiered metadata contains offset
> > ranges
> > > > and
> > > > > > > epoch
> > > > > > > > > > > history. Those should be enough for reconciliation
> > purposes.
> > > > > > > > > > >
> > > > > > > > > > > 1003.3 Having just a serverEndpoint string is probably
> > not
> > > > enough.
> > > > > > > > > > > Connecting to a Kafka cluster may need various security
> > > > > > > credentials.
> > > > > > > > We
> > > > > > > > > > can
> > > > > > > > > > > make RLMM configurable and pass in the properties
> > through the
> > > > > > > > > configure()
> > > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > > >
> > > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > > > log.retention.minutes,
> > > > > > > > > and
> > > > > > > > > > > added log.retention.ms later. If we are adding a new
> > > > > > > configuration,
> > > > > > > > ms
> > > > > > > > > > > level config alone is enough and is simpler. We can build
> > > > tools to
> > > > > > > > make
> > > > > > > > > > the
> > > > > > > > > > > configuration at different granularities easier. The
> > > > definition of
> > > > > > > > > > > log.retention.ms is "The number of milliseconds to keep
> > a
> > > > log file
> > > > > > > > > > before
> > > > > > > > > > > deleting it". The deletion is independent of whether
> > tiering
> > > > is
> > > > > > > > enabled
> > > > > > > > > > or
> > > > > > > > > > > not. If this changes to just the local portion of the
> > data,
> > > > we are
> > > > > > > > > > changing
> > > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > >
> > > > > > > > > > > > Thank you for the comments! Ying, Harsha and I
> > discussed
> > > > and put
> > > > > > > > our
> > > > > > > > > > > > comments below.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 1001. The KIP described a few scenarios of unclean
> > leader
> > > > > > > > elections.
> > > > > > > > > > This
> > > > > > > > > > > > is very useful, but I am wondering if this is the best
> > > > approach.
> > > > > > > My
> > > > > > > > > > > > understanding of the proposed approach is to allow the
> > new
> > > > > > > > (unclean)
> > > > > > > > > > leader
> > > > > > > > > > > > to take new messages immediately. While this increases
> > > > > > > > availability,
> > > > > > > > > it
> > > > > > > > > > > > creates the problem that there could be multiple
> > > > conflicting
> > > > > > > > segments
> > > > > > > > > > in
> > > > > > > > > > > > the remote store for the same offset range. This seems
> > to
> > > > make it
> > > > > > > > > > harder
> > > > > > > > > > > > for RLMM to determine which archived log segments
> > contain
> > > > the
> > > > > > > > correct
> > > > > > > > > > data.
> > > > > > > > > > > > For example, an archived log segment could at one time
> > be
> > > > the
> > > > > > > > correct
> > > > > > > > > > data,
> > > > > > > > > > > > but be changed to incorrect data after an unclean
> > leader
> > > > > > > election.
> > > > > > > > An
> > > > > > > > > > > > alternative approach is to let the unclean leader use
> > the
> > > > > > > archived
> > > > > > > > > > data as
> > > > > > > > > > > > the source of truth. So, when the new (unclean) leader
> > > > takes
> > > > > > > over,
> > > > > > > > it
> > > > > > > > > > first
> > > > > > > > > > > > reconciles the local data based on the archived data
> > before
> > > > > > > taking
> > > > > > > > > new
> > > > > > > > > > > > messages. This makes the job of RLMM a bit easier
> > since all
> > > > > > > > archived
> > > > > > > > > > data
> > > > > > > > > > > > are considered correct. This increases availability a
> > bit.
> > > > > > > However,
> > > > > > > > > > since
> > > > > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > > > > >
> > > > > > > > > > > > Firstly, We don't want to assume the remote storage is
> > more
> > > > > > > > reliable
> > > > > > > > > > than
> > > > > > > > > > > > Kafka. Kafka unclean leader election usually happens
> > when
> > > > there
> > > > > > > is
> > > > > > > > a
> > > > > > > > > > large
> > > > > > > > > > > > scale outage that impacts multiple racks (or even
> > multiple
> > > > > > > > > availability
> > > > > > > > > > > > zones). In such a case, the remote storage may be
> > > > unavailable or
> > > > > > > > > > unstable.
> > > > > > > > > > > > Pulling a large amount of data from the remote storage
> > to
> > > > > > > reconcile
> > > > > > > > > the
> > > > > > > > > > > > local data may also exacerbate the outage. With the
> > current
> > > > > > > design,
> > > > > > > > > > the new
> > > > > > > > > > > > leader can start working even when the remote storage
> > is
> > > > > > > > temporarily
> > > > > > > > > > > > unavailable.
> > > > > > > > > > > >
> > > > > > > > > > > > Secondly, it is not easier to implement the reconciling
> > > > logic at
> > > > > > > > the
> > > > > > > > > > leader
> > > > > > > > > > > > side. It can take a long time for the new leader to
> > > > download the
> > > > > > > > > remote
> > > > > > > > > > > > data and rebuild local producer id / leader epoch
> > > > information.
> > > > > > > > During
> > > > > > > > > > this
> > > > > > > > > > > > period, the leader cannot accept any requests from the
> > > > clients
> > > > > > > and
> > > > > > > > > > > > followers. We have to introduce a new state for the
> > > > leader, and a
> > > > > > > > new
> > > > > > > > > > error
> > > > > > > > > > > > code to let the clients / followers know what is
> > happening.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > > > RemoteStorageManager.
> > > > > > > > > > We
> > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all
> > other
> > > > > > > methods,
> > > > > > > > > we
> > > > > > > > > > pass
> > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > >
> > > > > > > > > > > > Nice catch, we can have the RemoteLogSegmentMetadata
> > for
> > > > > > > > > copyLogSegment
> > > > > > > > > > > > too.
> > > > > > > > > > > >
> > > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > inclusive
> > > > or
> > > > > > > > > exclusive?
> > > > > > > > > > > >
> > > > > > > > > > > > It is inclusive.
> > > > > > > > > > > >
> > > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > leaderEpoch
> > > > > > > history
> > > > > > > > > for
> > > > > > > > > > a
> > > > > > > > > > > > partition.
> > > > > > > > > > > >
> > > > > > > > > > > > Yes, updated the KIP with the new method.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 1002.4 Could you define the type of
> > > > RemoteLogSegmentContext?
> > > > > > > > > > > >
> > > > > > > > > > > > This is removed in the latest code and it is not
> > needed.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > >
> > > > > > > > > > > > 1003.1 I am not sure why we need both of the following
> > > > methods
> > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine them into
> > > > one that
> > > > > > > > > takes
> > > > > > > > > > in
> > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > >     RemoteLogSegmentId
> > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > topicPartition,
> > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > >
> > > > > > > > > > > > Good point, these can be merged for now. I guess we
> > needed
> > > > them
> > > > > > > in
> > > > > > > > > > earlier
> > > > > > > > > > > > version of the implementation but it is not needed now.
> > > > > > > > > > > >
> > > > > > > > > > > > 1003.2 There seems to be some inconsistencies in the
> > > > methods
> > > > > > > > below. I
> > > > > > > > > > am
> > > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the
> > > > other
> > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > >     void
> > putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > >     void
> > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > >
> > > > > > > > > > > > RLMM stores RemoteLogSegmentMetadata which is
> > identified by
> > > > > > > > > > > > RemoteLogsSegmentId. So, when it is added it takes
> > > > > > > > > > > > RemoteLogSegmentMetadata. `delete` operation needs only
> > > > > > > > > > RemoteLogsSegmentId
> > > > > > > > > > > > as RemoteLogSegmentMetadata can be identified with
> > > > > > > > > RemoteLogsSegmentId.
> > > > > > > > > > > >
> > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > > serverEndpoint), what
> > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > >
> > > > > > > > > > > > This can be used by RLMM implementation to connect to
> > the
> > > > local
> > > > > > > > Kafka
> > > > > > > > > > > > cluster. Incase of default  implementation, it is used
> > in
> > > > > > > > > initializing
> > > > > > > > > > > > kafka clients connecting to the local cluster.
> > > > > > > > > > > >
> > > > > > > > > > > > 1004. It would be useful to document how all the new
> > APIs
> > > > are
> > > > > > > being
> > > > > > > > > > used.
> > > > > > > > > > > > For example, when is
> > > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > being
> > > > > > > > > > set
> > > > > > > > > > > > and used? How are
> > > > > > > > > > > >
> > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > > > > > > > used?
> > > > > > > > > > > >
> > > > > > > > > > > > RLMM APIs are going through the changes and they
> > should be
> > > > ready
> > > > > > > > in a
> > > > > > > > > > few
> > > > > > > > > > > > days. I will update the KIP and the mail  thread once
> > they
> > > > are
> > > > > > > > ready.
> > > > > > > > > > > >
> > > > > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM
> > will
> > > > > > > > eventually
> > > > > > > > > > > > delete these segments by using RemoteStorageManager."
> > Which
> > > > > > > replica
> > > > > > > > > > does
> > > > > > > > > > > > this logic?
> > > > > > > > > > > >
> > > > > > > > > > > > This is a good point. When a topic is deleted, it will
> > not
> > > > have
> > > > > > > any
> > > > > > > > > > > > leader/followers to do the cleanup. We will have a
> > cleaner
> > > > agent
> > > > > > > > on a
> > > > > > > > > > > > single broker in the cluster to do this cleanup, we
> > plan
> > > > to add
> > > > > > > > that
> > > > > > > > > in
> > > > > > > > > > > > controller broker.
> > > > > > > > > > > >
> > > > > > > > > > > > 1006. "If there are any failures in removing remote log
> > > > segments
> > > > > > > > then
> > > > > > > > > > those
> > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > remote-log-segment-id)
> > > > > > > > > > from
> > > > > > > > > > > > that topic and clean them up from remote storage.  "
> > Not
> > > > sure if
> > > > > > > > it's
> > > > > > > > > > worth
> > > > > > > > > > > > the complexity of adding another topic. Could we just
> > > > retry?
> > > > > > > > > > > >
> > > > > > > > > > > > Sure, we can keep this simpler for now by logging an
> > error
> > > > after
> > > > > > > > > > retries.
> > > > > > > > > > > > We can give users a better way to process this in
> > future.
> > > > Oneway
> > > > > > > > can
> > > > > > > > > > be a
> > > > > > > > > > > > dead letter topic which can be configured by the user.
> > > > > > > > > > > >
> > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> > > > existing
> > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > >
> > > > > > > > > > > > We have 2 types of delayed operations waiting for 2
> > > > different
> > > > > > > > events.
> > > > > > > > > > > > DelayedFetch waits for new messages from producers.
> > > > > > > > > DelayedRemoteFetch
> > > > > > > > > > > > waits for the remote-storage-read-task to finish. When
> > > > either of
> > > > > > > > the
> > > > > > > > > 2
> > > > > > > > > > > > events happens, we only want to notify one type of the
> > > > delayed
> > > > > > > > > > operations.
> > > > > > > > > > > > It would be inefficient to put 2 types of delayed
> > > > operations in
> > > > > > > one
> > > > > > > > > > > > purgatory, as the tryComplete() methods of the delayed
> > > > operations
> > > > > > > > can
> > > > > > > > > > be
> > > > > > > > > > > > triggered by irrelevant events.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > > remote.log.retention.minutes,
> > > > > > > > > > > > remote.log.retention.hours: It seems that we just need
> > the
> > > > ms
> > > > > > > one.
> > > > > > > > > > Also,
> > > > > > > > > > > > are we changing the meaning of existing config
> > > > log.retention.ms
> > > > > > > to
> > > > > > > > > > mean
> > > > > > > > > > > > the
> > > > > > > > > > > > local retention? For backward compatibility, it's
> > better
> > > > to not
> > > > > > > > > change
> > > > > > > > > > the
> > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > >
> > > > > > > > > > > > We agree that we only need remote.log.retention.ms.
> > But,
> > > > the
> > > > > > > > > existing
> > > > > > > > > > > > Kafka
> > > > > > > > > > > > configuration
> > > > > > > > > > > > has 3 properties (log.retention.ms,
> > log.retention.minutes,
> > > > > > > > > > > > log.retention.hours). We just
> > > > > > > > > > > > want to keep consistent with the existing properties.
> > > > > > > > > > > > Existing log.retention.xxxx config is about log
> > retention
> > > > in
> > > > > > > > broker’s
> > > > > > > > > > > > storage which is local. It should be easy for users to
> > > > configure
> > > > > > > > > > partition
> > > > > > > > > > > > storage with local retention and remote retention
> > based on
> > > > their
> > > > > > > > > usage.
> > > > > > > > > > > >
> > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic
> > > > level?
> > > > > > > > > > > >
> > > > > > > > > > > > We can introduce topic level config for the same
> > remote.log
> > > > > > > > settings.
> > > > > > > > > > User
> > > > > > > > > > > > can set the desired config while creating the topic.
> > > > > > > > > > > > remote.log.storage.enable property is not allowed to be
> > > > updated
> > > > > > > > after
> > > > > > > > > > the
> > > > > > > > > > > > topic is created. Other remote.log.* properties can be
> > > > modified.
> > > > > > > We
> > > > > > > > > > will
> > > > > > > > > > > > support flipping remote.log.storage.enable in next
> > > > versions.
> > > > > > > > > > > >
> > > > > > > > > > > > 1009. It would be useful to list all limitations in a
> > > > separate
> > > > > > > > > section:
> > > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic
> > from
> > > > delete
> > > > > > > > to
> > > > > > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > > > > > >
> > > > > > > > > > > > +1 to have limitations in a separate section. We will
> > > > update the
> > > > > > > > KIP
> > > > > > > > > > with
> > > > > > > > > > > > that.
> > > > > > > > > > > > Topic  created with effective value for
> > remote.log.enabled
> > > > as
> > > > > > > true,
> > > > > > > > > > can not
> > > > > > > > > > > > change its retention policy from delete to compact.
> > > > > > > > > > > >
> > > > > > > > > > > > 1010. Thanks for performance numbers. Are those with
> > > > RocksDB as
> > > > > > > the
> > > > > > > > > > cache?
> > > > > > > > > > > >
> > > > > > > > > > > > No, We have not yet added RocksDB support. This is
> > based on
> > > > > > > > in-memory
> > > > > > > > > > map
> > > > > > > > > > > > representation. We will add that support and update
> > this
> > > > thread
> > > > > > > > after
> > > > > > > > > > > > updating the KIP with the numbers.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Satish.
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <
> > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> > below.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1000. Regarding Colin's question on querying the
> > metadata
> > > > > > > > directly
> > > > > > > > > > in the
> > > > > > > > > > > > > remote block store. One issue is that not all block
> > > > stores
> > > > > > > offer
> > > > > > > > > the
> > > > > > > > > > > > needed
> > > > > > > > > > > > > api to query the metadata. For example, S3 only
> > offers
> > > > an api
> > > > > > > to
> > > > > > > > > list
> > > > > > > > > > > > > objects under a prefix and this api has the eventual
> > > > > > > consistency
> > > > > > > > > > > > semantic.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1001. The KIP described a few scenarios of unclean
> > leader
> > > > > > > > > elections.
> > > > > > > > > > This
> > > > > > > > > > > > > is very useful, but I am wondering if this is the
> > best
> > > > > > > approach.
> > > > > > > > My
> > > > > > > > > > > > > understanding of the proposed approach is to allow
> > the
> > > > new
> > > > > > > > > (unclean)
> > > > > > > > > > > > leader
> > > > > > > > > > > > > to take new messages immediately. While this
> > increases
> > > > > > > > > availability,
> > > > > > > > > > it
> > > > > > > > > > > > > creates the problem that there could be multiple
> > > > conflicting
> > > > > > > > > > segments in
> > > > > > > > > > > > > the remote store for the same offset range. This
> > seems
> > > > to make
> > > > > > > it
> > > > > > > > > > harder
> > > > > > > > > > > > > for RLMM to determine which archived log segments
> > > > contain the
> > > > > > > > > correct
> > > > > > > > > > > > data.
> > > > > > > > > > > > > For example, an archived log segment could at one
> > time
> > > > be the
> > > > > > > > > correct
> > > > > > > > > > > > data,
> > > > > > > > > > > > > but be changed to incorrect data after an unclean
> > leader
> > > > > > > > election.
> > > > > > > > > An
> > > > > > > > > > > > > alternative approach is to let the unclean leader
> > use the
> > > > > > > > archived
> > > > > > > > > > data
> > > > > > > > > > > > as
> > > > > > > > > > > > > the source of truth. So, when the new (unclean)
> > leader
> > > > takes
> > > > > > > > over,
> > > > > > > > > it
> > > > > > > > > > > > first
> > > > > > > > > > > > > reconciles the local data based on the archived data
> > > > before
> > > > > > > > taking
> > > > > > > > > > new
> > > > > > > > > > > > > messages. This makes the job of RLMM a bit easier
> > since
> > > > all
> > > > > > > > > archived
> > > > > > > > > > data
> > > > > > > > > > > > > are considered correct. This increases availability a
> > > > bit.
> > > > > > > > However,
> > > > > > > > > > since
> > > > > > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > > > > RemoteStorageManager. We
> > > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all
> > > > other
> > > > > > > > methods,
> > > > > > > > > > we
> > > > > > > > > > > > pass
> > > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > > > inclusive or
> > > > > > > > > > exclusive?
> > > > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > > > leaderEpoch
> > > > > > > > history
> > > > > > > > > > for a
> > > > > > > > > > > > > partition.
> > > > > > > > > > > > > 1002.4 Could you define the type of
> > > > RemoteLogSegmentContext?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > > > 1003.1 I am not sure why we need both of the
> > following
> > > > methods
> > > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine them
> > into
> > > > one
> > > > > > > that
> > > > > > > > > > takes in
> > > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > > >     RemoteLogSegmentId
> > > > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > > > topicPartition,
> > > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > 1003.2 There seems to be some inconsistencies in the
> > > > methods
> > > > > > > > below.
> > > > > > > > > > I am
> > > > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and
> > the
> > > > other
> > > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > > >     void
> > putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > > >     void
> > > > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > > serverEndpoint),
> > > > > > > what
> > > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1004. It would be useful to document how all the new
> > > > APIs are
> > > > > > > > being
> > > > > > > > > > used.
> > > > > > > > > > > > > For example, when is
> > > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > > > being set
> > > > > > > > > > > > > and used? How are
> > > > > > > > > > > > >
> > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > > > > > > being
> > > > > > > > > > used?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM
> > > > will
> > > > > > > > > eventually
> > > > > > > > > > > > > delete these segments by using RemoteStorageManager."
> > > > Which
> > > > > > > > replica
> > > > > > > > > > does
> > > > > > > > > > > > > this logic?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1006. "If there are any failures in removing remote
> > log
> > > > > > > segments
> > > > > > > > > then
> > > > > > > > > > > > those
> > > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > > remote-log-segment-id)
> > > > > > > > > > from
> > > > > > > > > > > > > that topic and clean them up from remote storage.  "
> > Not
> > > > sure
> > > > > > > if
> > > > > > > > > it's
> > > > > > > > > > > > worth
> > > > > > > > > > > > > the complexity of adding another topic. Could we just
> > > > retry?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> > > > existing
> > > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > > remote.log.retention.minutes,
> > > > > > > > > > > > > remote.log.retention.hours: It seems that we just
> > need
> > > > the ms
> > > > > > > > one.
> > > > > > > > > > Also,
> > > > > > > > > > > > > are we changing the meaning of existing config
> > > > > > > log.retention.ms
> > > > > > > > to
> > > > > > > > > > mean
> > > > > > > > > > > > > the
> > > > > > > > > > > > > local retention? For backward compatibility, it's
> > better
> > > > to not
> > > > > > > > > > change
> > > > > > > > > > > > the
> > > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the
> > topic
> > > > level?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1009. It would be useful to list all limitations in a
> > > > separate
> > > > > > > > > > section:
> > > > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic
> > > > from
> > > > > > > delete
> > > > > > > > > to
> > > > > > > > > > > > > compact and vice versa allowed when tiering is
> > enabled?
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1010. Thanks for performance numbers. Are those with
> > > > RocksDB as
> > > > > > > > the
> > > > > > > > > > > > cache?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <
> > > > harsha.ch@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > >                Thats not what we said in the
> > previous
> > > > email.
> > > > > > > > RLMM
> > > > > > > > > > is
> > > > > > > > > > > > > > pluggable storage and by running numbers even 1PB
> > data
> > > > you do
> > > > > > > > not
> > > > > > > > > > need
> > > > > > > > > > > > > more
> > > > > > > > > > > > > > than 10GB local storage.
> > > > > > > > > > > > > > If in future this becomes a blocker for any users
> > we
> > > > can
> > > > > > > > revisit
> > > > > > > > > > but
> > > > > > > > > > > > this
> > > > > > > > > > > > > > does not warrant another implementation at this
> > point
> > > > to push
> > > > > > > > the
> > > > > > > > > > data
> > > > > > > > > > > > to
> > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > We can ofcourse implement another RLMM that is
> > > > optional for
> > > > > > > > users
> > > > > > > > > > to
> > > > > > > > > > > > > > configure to push to remote. But that doesn't need
> > to
> > > > be
> > > > > > > > > addressed
> > > > > > > > > > in
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Harsha
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe <
> > > > > > > > cmccabe@apache.org
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Ying,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for the response.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > It sounds like you agree that storing the
> > metadata
> > > > in the
> > > > > > > > > remote
> > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > would be a better design overall.  Given that
> > that's
> > > > true,
> > > > > > > is
> > > > > > > > > > there
> > > > > > > > > > > > any
> > > > > > > > > > > > > > > reason to include the worse implementation based
> > on
> > > > > > > RocksDB?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Choosing a long-term metadata store is not
> > something
> > > > that
> > > > > > > we
> > > > > > > > > > should
> > > > > > > > > > > > do
> > > > > > > > > > > > > > > lightly.  It can take users years to migrate from
> > > > metadata
> > > > > > > > > store
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > other.  I also don't think it's realistic or
> > > > desirable for
> > > > > > > > > users
> > > > > > > > > > to
> > > > > > > > > > > > > write
> > > > > > > > > > > > > > > their own metadata stores.  Even assuming that
> > they
> > > > could
> > > > > > > do
> > > > > > > > a
> > > > > > > > > > good
> > > > > > > > > > > > job
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > this, it would create huge fragmentation in the
> > Kafka
> > > > > > > > > ecosystem.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng wrote:
> > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Satish and I are still discussing some details
> > > > about how
> > > > > > > to
> > > > > > > > > > handle
> > > > > > > > > > > > > > > > transactions / producer ids. Satish is going to
> > > > make some
> > > > > > > > > minor
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > RLMM API and other parts. Other than that, we
> > have
> > > > > > > finished
> > > > > > > > > > > > updating
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I agree with Colin that the current design of
> > using
> > > > > > > rocksDB
> > > > > > > > > is
> > > > > > > > > > not
> > > > > > > > > > > > > > > > optimal. But this design is simple and should
> > work
> > > > for
> > > > > > > > almost
> > > > > > > > > > all
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > existing Kafka users. RLMM is a plugin. Users
> > can
> > > > replace
> > > > > > > > > > rocksDB
> > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > their own RLMM implementation, if needed. So, I
> > > > think we
> > > > > > > > can
> > > > > > > > > > keep
> > > > > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > > > for now. What do you think?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Ying
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> > > > > > > jun@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks for the update. It's good to see the
> > > > progress on
> > > > > > > > > this.
> > > > > > > > > > > > > Please
> > > > > > > > > > > > > > > let us
> > > > > > > > > > > > > > > > > know when you are done updating the KIP wiki.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng
> > > > > > > > > > > > <yingz@uber.com.invalid
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Satish and I have added more design
> > details in
> > > > the
> > > > > > > KIP,
> > > > > > > > > > > > including
> > > > > > > > > > > > > > > how to
> > > > > > > > > > > > > > > > > > keep consistency between replicas
> > (especially
> > > > when
> > > > > > > > there
> > > > > > > > > is
> > > > > > > > > > > > > > > leadership
> > > > > > > > > > > > > > > > > > changes / log truncations) and new
> > metrics. We
> > > > also
> > > > > > > > made
> > > > > > > > > > some
> > > > > > > > > > > > > other
> > > > > > > > > > > > > > > minor
> > > > > > > > > > > > > > > > > > changes in the doc. We will finish the KIP
> > > > changes in
> > > > > > > > the
> > > > > > > > > > next
> > > > > > > > > > > > > > > couple of
> > > > > > > > > > > > > > > > > > days. We will let you know when we are
> > done.
> > > > Most of
> > > > > > > > the
> > > > > > > > > > > > changes
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > already updated to the wiki KIP. You can
> > take
> > > > a look.
> > > > > > > > But
> > > > > > > > > > it's
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > As for the implementation, the code is
> > mostly
> > > > done
> > > > > > > and
> > > > > > > > we
> > > > > > > > > > > > already
> > > > > > > > > > > > > > had
> > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > feature tests / system tests. I have added
> > the
> > > > > > > > > performance
> > > > > > > > > > test
> > > > > > > > > > > > > > > results
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the KIP. However the recent design changes
> > > > (e.g.
> > > > > > > leader
> > > > > > > > > > epoch
> > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > management / log truncation / some of the
> > new
> > > > > > > metrics)
> > > > > > > > > > have not
> > > > > > > > > > > > > > been
> > > > > > > > > > > > > > > > > > implemented yet. It will take about 2 weeks
> > > > for us to
> > > > > > > > > > implement
> > > > > > > > > > > > > > > after you
> > > > > > > > > > > > > > > > > > review and agree with those design changes.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun Rao <
> > > > > > > > jun@confluent.io
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Any new updates on the KIP? This feature
> > is
> > > > one of
> > > > > > > > the
> > > > > > > > > > most
> > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > most requested features in Apache Kafka
> > > > right now.
> > > > > > > It
> > > > > > > > > > would
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > helpful
> > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > we can make sustained progress on this.
> > > > Could you
> > > > > > > > share
> > > > > > > > > > how
> > > > > > > > > > > > far
> > > > > > > > > > > > > > > along
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > the design/implementation right now? Is
> > there
> > > > > > > > anything
> > > > > > > > > > that
> > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > As for "transactional support" and
> > "follower
> > > > > > > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > > > > > > further comments from me as long as the
> > > > producer
> > > > > > > > state
> > > > > > > > > > and
> > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > be restored properly from the object
> > store
> > > > when
> > > > > > > > needed.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM Satish
> > > > Duggana <
> > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > We did not want to add many
> > implementation
> > > > > > > details
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > But we
> > > > > > > > > > > > > > > > > > > > decided to add them in the KIP as
> > appendix
> > > > or
> > > > > > > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > > > > > > follower fetch protocol) to describe
> > the
> > > > flow
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > main
> > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > > That will answer most of the queries. I
> > > > will
> > > > > > > update
> > > > > > > > > on
> > > > > > > > > > this
> > > > > > > > > > > > > > mail
> > > > > > > > > > > > > > > > > > > > thread when the respective sections are
> > > > updated.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM
> > Alexandre
> > > > Dupriez
> > > > > > > > > > > > > > > > > > > > <al...@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > A couple of questions specific to the
> > > > section
> > > > > > > > > > "Follower
> > > > > > > > > > > > > > > > > > > > > Requests/Replication", pages 16:17
> > in the
> > > > > > > design
> > > > > > > > > > document
> > > > > > > > > > > > > > [1].
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900. It is mentioned that followers
> > fetch
> > > > > > > > auxiliary
> > > > > > > > > > > > states
> > > > > > > > > > > > > > > from the
> > > > > > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900.a Does the consistency model of
> > the
> > > > > > > external
> > > > > > > > > > storage
> > > > > > > > > > > > > > > impacts
> > > > > > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > > > > > > of leader epochs and other auxiliary
> > > > data?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900.b What are the benefits of using
> > a
> > > > > > > mechanism
> > > > > > > > to
> > > > > > > > > > store
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > > > > > the leader epochs which is different
> > > > from other
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > tiered segments? What are the
> > benefits of
> > > > > > > > > retrieving
> > > > > > > > > > this
> > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > on-demand from the follower rather
> > than
> > > > relying
> > > > > > > > on
> > > > > > > > > > > > > > propagation
> > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > topic __remote_log_metadata? What
> > are the
> > > > > > > > > advantages
> > > > > > > > > > over
> > > > > > > > > > > > > > > using a
> > > > > > > > > > > > > > > > > > > > > dedicated control structure (e.g. a
> > new
> > > > record
> > > > > > > > > type)
> > > > > > > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > > > > > > this topic? Since in the document,
> > > > different
> > > > > > > > > control
> > > > > > > > > > > > paths
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > operating in the system, how are the
> > > > metadata
> > > > > > > > > stored
> > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > __remote_log_metadata [which also
> > > > include the
> > > > > > > > epoch
> > > > > > > > > > of
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > which offloaded a segment] and the
> > remote
> > > > > > > > auxiliary
> > > > > > > > > > > > states,
> > > > > > > > > > > > > > > kept in
> > > > > > > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900.c A follower can encounter an
> > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > > > > > > this in response to a Fetch or
> > > > > > > > OffsetForLeaderEpoch
> > > > > > > > > > > > > request?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900.d What happens if, after a
> > follower
> > > > > > > > encountered
> > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE
> > response,
> > > > its
> > > > > > > > > > attempts to
> > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > leader epochs fail (for instance,
> > > > because the
> > > > > > > > > remote
> > > > > > > > > > > > > storage
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > temporarily unavailable)? Does the
> > > > follower
> > > > > > > > > > fallbacks to
> > > > > > > > > > > > a
> > > > > > > > > > > > > > mode
> > > > > > > > > > > > > > > > > where
> > > > > > > > > > > > > > > > > > > > > it ignores tiered segments, and
> > applies
> > > > > > > > truncation
> > > > > > > > > > using
> > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > > > available information? What happens
> > when
> > > > access
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > > is restored? How is the replica
> > lineage
> > > > > > > inferred
> > > > > > > > by
> > > > > > > > > > the
> > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > epochs reconciled with the follower's
> > > > replica
> > > > > > > > > > lineage,
> > > > > > > > > > > > > which
> > > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > > > > evolved? Does the follower remember
> > > > fetching
> > > > > > > > > > auxiliary
> > > > > > > > > > > > > states
> > > > > > > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > > > > > > in the past and attempt
> > reconciliation?
> > > > Is
> > > > > > > there
> > > > > > > > a
> > > > > > > > > > plan
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > offer
> > > > > > > > > > > > > > > > > > > > > different strategies in this
> > scenario,
> > > > > > > > configurable
> > > > > > > > > > via
> > > > > > > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900.e Is the leader epoch cache
> > > > offloaded with
> > > > > > > > > every
> > > > > > > > > > > > > segment?
> > > > > > > > > > > > > > > Or
> > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > a new checkpoint is detected? If that
> > > > > > > information
> > > > > > > > > is
> > > > > > > > > > not
> > > > > > > > > > > > > > always
> > > > > > > > > > > > > > > > > > > > > offloaded to avoid duplicating data,
> > how
> > > > does
> > > > > > > the
> > > > > > > > > > remote
> > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > > satisfy the request to retrieve it?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900.f Since the leader epoch cache
> > > > covers the
> > > > > > > > > entire
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > > > > > > what happens if, after a leader epoch
> > > > cache
> > > > > > > file
> > > > > > > > is
> > > > > > > > > > > > > offloaded
> > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > given segment, the local epoch cache
> > is
> > > > > > > truncated
> > > > > > > > > > [not
> > > > > > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > a range of offset included in tiered
> > > > segments]?
> > > > > > > > How
> > > > > > > > > > are
> > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > local leader epoch caches kept
> > > > consistent?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900.g Consumer can also use leader
> > > > epochs (e.g.
> > > > > > > > to
> > > > > > > > > > enable
> > > > > > > > > > > > > > > fencing
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > protect against stale leaders). What
> > > > > > > differences
> > > > > > > > > > would
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > between consumer and follower
> > fetches?
> > > > > > > > Especially,
> > > > > > > > > > would
> > > > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > > > > > also fetch leader epoch information
> > from
> > > > the
> > > > > > > > remote
> > > > > > > > > > > > > storage?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 900.h Assume a newly elected leader
> > of a
> > > > > > > > > > topic-partition
> > > > > > > > > > > > > > > detects
> > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > recent segments are available in the
> > > > external
> > > > > > > > > > storage,
> > > > > > > > > > > > with
> > > > > > > > > > > > > > > epochs
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > its local epoch. Does it ignore these
> > > > segments
> > > > > > > > and
> > > > > > > > > > their
> > > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > > > epoch-to-offset vectors? Or try to
> > > > reconstruct
> > > > > > > > its
> > > > > > > > > > local
> > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > lineage based on the data remotely
> > > > available?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > >
> > https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55, Satish
> > > > Duggana <
> > > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > > > > Please let us know if you have any
> > > > comments
> > > > > > > on
> > > > > > > > > > > > > > "transactional
> > > > > > > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > > > > > > and "follower requests/replication"
> > > > mentioned
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > wiki.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25 PM
> > Satish
> > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks Jun for your comments.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >100. It would be useful to
> > provide
> > > > more
> > > > > > > > > details
> > > > > > > > > > on
> > > > > > > > > > > > how
> > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard
> > to
> > > > really
> > > > > > > > > assess
> > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > > > > apis are sufficient/redundant. A few
> > > > examples
> > > > > > > > below.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > We will update the wiki and let
> > you
> > > > know.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >100.1 deleteRecords seems to
> > only
> > > > advance
> > > > > > > > the
> > > > > > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > Log. How does that trigger the
> > deletion of
> > > > remote
> > > > > > > > log
> > > > > > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > RLMTask for leader partition
> > > > periodically
> > > > > > > > > checks
> > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > remote log segments earlier to
> > > > > > > logStartOffset
> > > > > > > > > > and the
> > > > > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > > > > > remote log segment metadata and
> > data
> > > > are
> > > > > > > > > deleted
> > > > > > > > > > by
> > > > > > > > > > > > > using
> > > > > > > > > > > > > > > RLMM
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >100.2 stopReplica with deletion
> > is
> > > > used
> > > > > > > in 2
> > > > > > > > > > cases
> > > > > > > > > > > > (a)
> > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > reassignment; (b) topic deletion. We
> > only
> > > > want to
> > > > > > > > > > delete
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > > > metadata in the second case. Also, in
> > the
> > > > second
> > > > > > > > > case,
> > > > > > > > > > who
> > > > > > > >
> >


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

Posted by Kowshik Prakasam <kp...@confluent.io>.
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
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' 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 <sa...@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 <ju...@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 <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > >
> > > Satish.
> > >
> > >
> > > On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana <
> satish.duggana@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > > >
> > > >
> > > > Satish.
> > > >
> > > >
> > > > On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng <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 <ju...@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
> <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
> > > > > > > unclean-leader-election.
> > > > > > > 2) Since the user set local retention time (or local retention
> > > bytes), I
> > > > > > > think we are expected to
> > > > > > > keep that much local data when possible (avoid truncating all
> the
> > > local
> > > > > > > data). But, as you said,
> > > > > > > unclean leader elections are very rare, this may not be a big
> > > problem.
> > > > > > >
> > > > > > > The current design uses the leader broker as source-of-truth.
> This
> > > is
> > > > > > > consistent with the
> > > > > > > existing Kafka behavior.
> > > > > > >
> > > > > > > By using remote storage as the source-of-truth, the follower
> logic
> > > can
> > > > > > be a
> > > > > > > little simpler,
> > > > > > > but the leader logic is going to be more complex. Overall, I
> don't
> > > see
> > > > > > > there many benefits
> > > > > > > of using remote storage as the source-of-truth.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the reply.
> > > > > > > >
> > > > > > > > 1001. In your example, I was thinking that you could just
> > > download the
> > > > > > > > latest leader epoch from the object store. After that you
> know
> > > the
> > > > > > leader
> > > > > > > > should end with offset 1100. The leader will delete all its
> > > local data
> > > > > > > > before offset 1000 and start accepting new messages at offset
> > > 1100.
> > > > > > > > Consumer requests for messages before offset 1100 will be
> served
> > > from
> > > > > > the
> > > > > > > > object store. The benefit with this approach is that it's
> > > simpler to
> > > > > > > reason
> > > > > > > > about who is the source of truth. The downside is slightly
> > > increased
> > > > > > > > unavailability window during unclean leader election. Since
> > > unclean
> > > > > > > leader
> > > > > > > > elections are rare, I am not sure if this is a big concern.
> > > > > > > >
> > > > > > > > 1008. Yes, I think introducing sth like local.retention.ms
> > > seems more
> > > > > > > > consistent.
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > > > > > satish.duggana@gmail.com
> > > > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > HI Jun,
> > > > > > > > > Thanks for your comments. We put our inline replies below.
> > > > > > > > >
> > > > > > > > > 1001. I was thinking that you could just use the tiered
> > > metadata to
> > > > > > do
> > > > > > > > the
> > > > > > > > > reconciliation. The tiered metadata contains offset ranges
> and
> > > epoch
> > > > > > > > > history. Those should be enough for reconciliation
> purposes.
> > > > > > > > >
> > > > > > > > > If we use remote storage as the source-of-truth during
> > > > > > > > > unclean-leader-election, it's possible that after
> > > reconciliation the
> > > > > > > > > remote storage will have more recent data than the new
> > > leader's local
> > > > > > > > > storage. For example, the new leader's latest message is
> > > offset 1000,
> > > > > > > > > while the remote storage has message 1100. In such a case,
> the
> > > new
> > > > > > > > > leader will have to download the messages from 1001 to
> 1100,
> > > before
> > > > > > > > > accepting new messages from producers. Otherwise, there
> would
> > > be a
> > > > > > gap
> > > > > > > > > in the local data between 1000 and 1101.
> > > > > > > > >
> > > > > > > > > Moreover, with the current design, leader epoch history is
> > > stored in
> > > > > > > > > remote storage, rather than the metadata topic. We did
> consider
> > > > > > saving
> > > > > > > > > epoch history in remote segment metadata. But the concern
> is
> > > that
> > > > > > > > > there is currently no limit for the epoch history size.
> > > > > > Theoretically,
> > > > > > > > > if a user has a very long remote retention time and there
> are
> > > very
> > > > > > > > > frequent leadership changes, the leader epoch history can
> > > become too
> > > > > > > > > long to fit into a regular Kafka message.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 1003.3 Having just a serverEndpoint string is probably not
> > > enough.
> > > > > > > > > Connecting to a Kafka cluster may need various security
> > > credentials.
> > > > > > We
> > > > > > > > can
> > > > > > > > > make RLMM configurable and pass in the properties through
> the
> > > > > > > configure()
> > > > > > > > > method. Ditto for RSM.
> > > > > > > > >
> > > > > > > > > RLMM and  RSM are already configurable and they take
> > > properties which
> > > > > > > > > start with "remote.log.metadata." and "remote.log.storage."
> > > > > > > > > respectively and a few others. We have listener-name as the
> > > config
> > > > > > for
> > > > > > > > > RLMM and other properties(like security) can be sent as you
> > > > > > suggested.
> > > > > > > > > We will update the KIP with the details.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 1008.1 We started with log.retention.hours and
> > > log.retention.minutes,
> > > > > > > and
> > > > > > > > > added log.retention.ms later. If we are adding a new
> > > configuration,
> > > > > > ms
> > > > > > > > > level config alone is enough and is simpler. We can build
> > > tools to
> > > > > > make
> > > > > > > > the
> > > > > > > > > configuration at different granularities easier. The
> > > definition of
> > > > > > > > > log.retention.ms is "The number of milliseconds to keep a
> log
> > > file
> > > > > > > > before
> > > > > > > > > deleting it". The deletion is independent of whether
> tiering is
> > > > > > enabled
> > > > > > > > or
> > > > > > > > > not. If this changes to just the local portion of the
> data, we
> > > are
> > > > > > > > changing
> > > > > > > > > the meaning of an existing configuration.
> > > > > > > > >
> > > > > > > > > We are fine with either way. We can go with
> log.retention.xxxx
> > > as the
> > > > > > > > > effective log retention instead of local log retention.
> With
> > > this
> > > > > > > > > convention, we need to introduce  local.log.retention
> instead
> > > of
> > > > > > > > > remote.log.retention.ms that we proposed. If
> log.retention.ms
> > > as -1
> > > > > > > > > then remote retention is also considered as unlimited but
> user
> > > should
> > > > > > > > > be able to set the local.retention.ms.
> > > > > > > > > So, we need to introduce local.log.retention.ms and
> > > > > > > > > local.log.retention.bytes which should  always  be <=
> > > > > > > > > log.retention.ms/bytes respectively.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > > > >
> > > > > > > > > > Hi, Satish,
> > > > > > > > > >
> > > > > > > > > > Thanks for the reply. A few quick comments below.
> > > > > > > > > >
> > > > > > > > > > 1001. I was thinking that you could just use the tiered
> > > metadata to
> > > > > > > do
> > > > > > > > > the
> > > > > > > > > > reconciliation. The tiered metadata contains offset
> ranges
> > > and
> > > > > > epoch
> > > > > > > > > > history. Those should be enough for reconciliation
> purposes.
> > > > > > > > > >
> > > > > > > > > > 1003.3 Having just a serverEndpoint string is probably
> not
> > > enough.
> > > > > > > > > > Connecting to a Kafka cluster may need various security
> > > > > > credentials.
> > > > > > > We
> > > > > > > > > can
> > > > > > > > > > make RLMM configurable and pass in the properties
> through the
> > > > > > > > configure()
> > > > > > > > > > method. Ditto for RSM.
> > > > > > > > > >
> > > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > > log.retention.minutes,
> > > > > > > > and
> > > > > > > > > > added log.retention.ms later. If we are adding a new
> > > > > > configuration,
> > > > > > > ms
> > > > > > > > > > level config alone is enough and is simpler. We can build
> > > tools to
> > > > > > > make
> > > > > > > > > the
> > > > > > > > > > configuration at different granularities easier. The
> > > definition of
> > > > > > > > > > log.retention.ms is "The number of milliseconds to keep
> a
> > > log file
> > > > > > > > > before
> > > > > > > > > > deleting it". The deletion is independent of whether
> tiering
> > > is
> > > > > > > enabled
> > > > > > > > > or
> > > > > > > > > > not. If this changes to just the local portion of the
> data,
> > > we are
> > > > > > > > > changing
> > > > > > > > > > the meaning of an existing configuration.
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Jun,
> > > > > > > > > > >
> > > > > > > > > > > Thank you for the comments! Ying, Harsha and I
> discussed
> > > and put
> > > > > > > our
> > > > > > > > > > > comments below.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 1001. The KIP described a few scenarios of unclean
> leader
> > > > > > > elections.
> > > > > > > > > This
> > > > > > > > > > > is very useful, but I am wondering if this is the best
> > > approach.
> > > > > > My
> > > > > > > > > > > understanding of the proposed approach is to allow the
> new
> > > > > > > (unclean)
> > > > > > > > > leader
> > > > > > > > > > > to take new messages immediately. While this increases
> > > > > > > availability,
> > > > > > > > it
> > > > > > > > > > > creates the problem that there could be multiple
> > > conflicting
> > > > > > > segments
> > > > > > > > > in
> > > > > > > > > > > the remote store for the same offset range. This seems
> to
> > > make it
> > > > > > > > > harder
> > > > > > > > > > > for RLMM to determine which archived log segments
> contain
> > > the
> > > > > > > correct
> > > > > > > > > data.
> > > > > > > > > > > For example, an archived log segment could at one time
> be
> > > the
> > > > > > > correct
> > > > > > > > > data,
> > > > > > > > > > > but be changed to incorrect data after an unclean
> leader
> > > > > > election.
> > > > > > > An
> > > > > > > > > > > alternative approach is to let the unclean leader use
> the
> > > > > > archived
> > > > > > > > > data as
> > > > > > > > > > > the source of truth. So, when the new (unclean) leader
> > > takes
> > > > > > over,
> > > > > > > it
> > > > > > > > > first
> > > > > > > > > > > reconciles the local data based on the archived data
> before
> > > > > > taking
> > > > > > > > new
> > > > > > > > > > > messages. This makes the job of RLMM a bit easier
> since all
> > > > > > > archived
> > > > > > > > > data
> > > > > > > > > > > are considered correct. This increases availability a
> bit.
> > > > > > However,
> > > > > > > > > since
> > > > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > > > >
> > > > > > > > > > > Firstly, We don't want to assume the remote storage is
> more
> > > > > > > reliable
> > > > > > > > > than
> > > > > > > > > > > Kafka. Kafka unclean leader election usually happens
> when
> > > there
> > > > > > is
> > > > > > > a
> > > > > > > > > large
> > > > > > > > > > > scale outage that impacts multiple racks (or even
> multiple
> > > > > > > > availability
> > > > > > > > > > > zones). In such a case, the remote storage may be
> > > unavailable or
> > > > > > > > > unstable.
> > > > > > > > > > > Pulling a large amount of data from the remote storage
> to
> > > > > > reconcile
> > > > > > > > the
> > > > > > > > > > > local data may also exacerbate the outage. With the
> current
> > > > > > design,
> > > > > > > > > the new
> > > > > > > > > > > leader can start working even when the remote storage
> is
> > > > > > > temporarily
> > > > > > > > > > > unavailable.
> > > > > > > > > > >
> > > > > > > > > > > Secondly, it is not easier to implement the reconciling
> > > logic at
> > > > > > > the
> > > > > > > > > leader
> > > > > > > > > > > side. It can take a long time for the new leader to
> > > download the
> > > > > > > > remote
> > > > > > > > > > > data and rebuild local producer id / leader epoch
> > > information.
> > > > > > > During
> > > > > > > > > this
> > > > > > > > > > > period, the leader cannot accept any requests from the
> > > clients
> > > > > > and
> > > > > > > > > > > followers. We have to introduce a new state for the
> > > leader, and a
> > > > > > > new
> > > > > > > > > error
> > > > > > > > > > > code to let the clients / followers know what is
> happening.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > > RemoteStorageManager.
> > > > > > > > > We
> > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all
> other
> > > > > > methods,
> > > > > > > > we
> > > > > > > > > pass
> > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > >
> > > > > > > > > > > Nice catch, we can have the RemoteLogSegmentMetadata
> for
> > > > > > > > copyLogSegment
> > > > > > > > > > > too.
> > > > > > > > > > >
> > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> inclusive
> > > or
> > > > > > > > exclusive?
> > > > > > > > > > >
> > > > > > > > > > > It is inclusive.
> > > > > > > > > > >
> > > > > > > > > > > 1002.3 It seems that we need an api to get the
> leaderEpoch
> > > > > > history
> > > > > > > > for
> > > > > > > > > a
> > > > > > > > > > > partition.
> > > > > > > > > > >
> > > > > > > > > > > Yes, updated the KIP with the new method.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 1002.4 Could you define the type of
> > > RemoteLogSegmentContext?
> > > > > > > > > > >
> > > > > > > > > > > This is removed in the latest code and it is not
> needed.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > >
> > > > > > > > > > > 1003.1 I am not sure why we need both of the following
> > > methods
> > > > > > > > > > > in RemoteLogMetadataManager. Could we combine them into
> > > one that
> > > > > > > > takes
> > > > > > > > > in
> > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > >     RemoteLogSegmentId
> getRemoteLogSegmentId(TopicPartition
> > > > > > > > > topicPartition,
> > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > >
> > > > > > > > > > > Good point, these can be merged for now. I guess we
> needed
> > > them
> > > > > > in
> > > > > > > > > earlier
> > > > > > > > > > > version of the implementation but it is not needed now.
> > > > > > > > > > >
> > > > > > > > > > > 1003.2 There seems to be some inconsistencies in the
> > > methods
> > > > > > > below. I
> > > > > > > > > am
> > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the
> > > other
> > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > >     void
> putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > >     void
> deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > >
> > > > > > > > > > > RLMM stores RemoteLogSegmentMetadata which is
> identified by
> > > > > > > > > > > RemoteLogsSegmentId. So, when it is added it takes
> > > > > > > > > > > RemoteLogSegmentMetadata. `delete` operation needs only
> > > > > > > > > RemoteLogsSegmentId
> > > > > > > > > > > as RemoteLogSegmentMetadata can be identified with
> > > > > > > > RemoteLogsSegmentId.
> > > > > > > > > > >
> > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > serverEndpoint), what
> > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > >
> > > > > > > > > > > This can be used by RLMM implementation to connect to
> the
> > > local
> > > > > > > Kafka
> > > > > > > > > > > cluster. Incase of default  implementation, it is used
> in
> > > > > > > > initializing
> > > > > > > > > > > kafka clients connecting to the local cluster.
> > > > > > > > > > >
> > > > > > > > > > > 1004. It would be useful to document how all the new
> APIs
> > > are
> > > > > > being
> > > > > > > > > used.
> > > > > > > > > > > For example, when is
> > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > being
> > > > > > > > > set
> > > > > > > > > > > and used? How are
> > > > > > > > > > >
> > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > > > > > > used?
> > > > > > > > > > >
> > > > > > > > > > > RLMM APIs are going through the changes and they
> should be
> > > ready
> > > > > > > in a
> > > > > > > > > few
> > > > > > > > > > > days. I will update the KIP and the mail  thread once
> they
> > > are
> > > > > > > ready.
> > > > > > > > > > >
> > > > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM
> will
> > > > > > > eventually
> > > > > > > > > > > delete these segments by using RemoteStorageManager."
> Which
> > > > > > replica
> > > > > > > > > does
> > > > > > > > > > > this logic?
> > > > > > > > > > >
> > > > > > > > > > > This is a good point. When a topic is deleted, it will
> not
> > > have
> > > > > > any
> > > > > > > > > > > leader/followers to do the cleanup. We will have a
> cleaner
> > > agent
> > > > > > > on a
> > > > > > > > > > > single broker in the cluster to do this cleanup, we
> plan
> > > to add
> > > > > > > that
> > > > > > > > in
> > > > > > > > > > > controller broker.
> > > > > > > > > > >
> > > > > > > > > > > 1006. "If there are any failures in removing remote log
> > > segments
> > > > > > > then
> > > > > > > > > those
> > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > and user can consume the events(which contain
> > > > > > > remote-log-segment-id)
> > > > > > > > > from
> > > > > > > > > > > that topic and clean them up from remote storage.  "
> Not
> > > sure if
> > > > > > > it's
> > > > > > > > > worth
> > > > > > > > > > > the complexity of adding another topic. Could we just
> > > retry?
> > > > > > > > > > >
> > > > > > > > > > > Sure, we can keep this simpler for now by logging an
> error
> > > after
> > > > > > > > > retries.
> > > > > > > > > > > We can give users a better way to process this in
> future.
> > > Oneway
> > > > > > > can
> > > > > > > > > be a
> > > > > > > > > > > dead letter topic which can be configured by the user.
> > > > > > > > > > >
> > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> > > existing
> > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > >
> > > > > > > > > > > We have 2 types of delayed operations waiting for 2
> > > different
> > > > > > > events.
> > > > > > > > > > > DelayedFetch waits for new messages from producers.
> > > > > > > > DelayedRemoteFetch
> > > > > > > > > > > waits for the remote-storage-read-task to finish. When
> > > either of
> > > > > > > the
> > > > > > > > 2
> > > > > > > > > > > events happens, we only want to notify one type of the
> > > delayed
> > > > > > > > > operations.
> > > > > > > > > > > It would be inefficient to put 2 types of delayed
> > > operations in
> > > > > > one
> > > > > > > > > > > purgatory, as the tryComplete() methods of the delayed
> > > operations
> > > > > > > can
> > > > > > > > > be
> > > > > > > > > > > triggered by irrelevant events.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > remote.log.retention.minutes,
> > > > > > > > > > > remote.log.retention.hours: It seems that we just need
> the
> > > ms
> > > > > > one.
> > > > > > > > > Also,
> > > > > > > > > > > are we changing the meaning of existing config
> > > log.retention.ms
> > > > > > to
> > > > > > > > > mean
> > > > > > > > > > > the
> > > > > > > > > > > local retention? For backward compatibility, it's
> better
> > > to not
> > > > > > > > change
> > > > > > > > > the
> > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > >
> > > > > > > > > > > We agree that we only need remote.log.retention.ms.
> But,
> > > the
> > > > > > > > existing
> > > > > > > > > > > Kafka
> > > > > > > > > > > configuration
> > > > > > > > > > > has 3 properties (log.retention.ms,
> log.retention.minutes,
> > > > > > > > > > > log.retention.hours). We just
> > > > > > > > > > > want to keep consistent with the existing properties.
> > > > > > > > > > > Existing log.retention.xxxx config is about log
> retention
> > > in
> > > > > > > broker’s
> > > > > > > > > > > storage which is local. It should be easy for users to
> > > configure
> > > > > > > > > partition
> > > > > > > > > > > storage with local retention and remote retention
> based on
> > > their
> > > > > > > > usage.
> > > > > > > > > > >
> > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic
> > > level?
> > > > > > > > > > >
> > > > > > > > > > > We can introduce topic level config for the same
> remote.log
> > > > > > > settings.
> > > > > > > > > User
> > > > > > > > > > > can set the desired config while creating the topic.
> > > > > > > > > > > remote.log.storage.enable property is not allowed to be
> > > updated
> > > > > > > after
> > > > > > > > > the
> > > > > > > > > > > topic is created. Other remote.log.* properties can be
> > > modified.
> > > > > > We
> > > > > > > > > will
> > > > > > > > > > > support flipping remote.log.storage.enable in next
> > > versions.
> > > > > > > > > > >
> > > > > > > > > > > 1009. It would be useful to list all limitations in a
> > > separate
> > > > > > > > section:
> > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic
> from
> > > delete
> > > > > > > to
> > > > > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > > > > >
> > > > > > > > > > > +1 to have limitations in a separate section. We will
> > > update the
> > > > > > > KIP
> > > > > > > > > with
> > > > > > > > > > > that.
> > > > > > > > > > > Topic  created with effective value for
> remote.log.enabled
> > > as
> > > > > > true,
> > > > > > > > > can not
> > > > > > > > > > > change its retention policy from delete to compact.
> > > > > > > > > > >
> > > > > > > > > > > 1010. Thanks for performance numbers. Are those with
> > > RocksDB as
> > > > > > the
> > > > > > > > > cache?
> > > > > > > > > > >
> > > > > > > > > > > No, We have not yet added RocksDB support. This is
> based on
> > > > > > > in-memory
> > > > > > > > > map
> > > > > > > > > > > representation. We will add that support and update
> this
> > > thread
> > > > > > > after
> > > > > > > > > > > updating the KIP with the numbers.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Satish.
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <
> jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the updated KIP. A few more comments
> below.
> > > > > > > > > > > >
> > > > > > > > > > > > 1000. Regarding Colin's question on querying the
> metadata
> > > > > > > directly
> > > > > > > > > in the
> > > > > > > > > > > > remote block store. One issue is that not all block
> > > stores
> > > > > > offer
> > > > > > > > the
> > > > > > > > > > > needed
> > > > > > > > > > > > api to query the metadata. For example, S3 only
> offers
> > > an api
> > > > > > to
> > > > > > > > list
> > > > > > > > > > > > objects under a prefix and this api has the eventual
> > > > > > consistency
> > > > > > > > > > > semantic.
> > > > > > > > > > > >
> > > > > > > > > > > > 1001. The KIP described a few scenarios of unclean
> leader
> > > > > > > > elections.
> > > > > > > > > This
> > > > > > > > > > > > is very useful, but I am wondering if this is the
> best
> > > > > > approach.
> > > > > > > My
> > > > > > > > > > > > understanding of the proposed approach is to allow
> the
> > > new
> > > > > > > > (unclean)
> > > > > > > > > > > leader
> > > > > > > > > > > > to take new messages immediately. While this
> increases
> > > > > > > > availability,
> > > > > > > > > it
> > > > > > > > > > > > creates the problem that there could be multiple
> > > conflicting
> > > > > > > > > segments in
> > > > > > > > > > > > the remote store for the same offset range. This
> seems
> > > to make
> > > > > > it
> > > > > > > > > harder
> > > > > > > > > > > > for RLMM to determine which archived log segments
> > > contain the
> > > > > > > > correct
> > > > > > > > > > > data.
> > > > > > > > > > > > For example, an archived log segment could at one
> time
> > > be the
> > > > > > > > correct
> > > > > > > > > > > data,
> > > > > > > > > > > > but be changed to incorrect data after an unclean
> leader
> > > > > > > election.
> > > > > > > > An
> > > > > > > > > > > > alternative approach is to let the unclean leader
> use the
> > > > > > > archived
> > > > > > > > > data
> > > > > > > > > > > as
> > > > > > > > > > > > the source of truth. So, when the new (unclean)
> leader
> > > takes
> > > > > > > over,
> > > > > > > > it
> > > > > > > > > > > first
> > > > > > > > > > > > reconciles the local data based on the archived data
> > > before
> > > > > > > taking
> > > > > > > > > new
> > > > > > > > > > > > messages. This makes the job of RLMM a bit easier
> since
> > > all
> > > > > > > > archived
> > > > > > > > > data
> > > > > > > > > > > > are considered correct. This increases availability a
> > > bit.
> > > > > > > However,
> > > > > > > > > since
> > > > > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > > > > >
> > > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > > > RemoteStorageManager. We
> > > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all
> > > other
> > > > > > > methods,
> > > > > > > > > we
> > > > > > > > > > > pass
> > > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > > inclusive or
> > > > > > > > > exclusive?
> > > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > > leaderEpoch
> > > > > > > history
> > > > > > > > > for a
> > > > > > > > > > > > partition.
> > > > > > > > > > > > 1002.4 Could you define the type of
> > > RemoteLogSegmentContext?
> > > > > > > > > > > >
> > > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > > 1003.1 I am not sure why we need both of the
> following
> > > methods
> > > > > > > > > > > > in RemoteLogMetadataManager. Could we combine them
> into
> > > one
> > > > > > that
> > > > > > > > > takes in
> > > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > > >     RemoteLogSegmentId
> > > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > > topicPartition,
> > > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > 1003.2 There seems to be some inconsistencies in the
> > > methods
> > > > > > > below.
> > > > > > > > > I am
> > > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and
> the
> > > other
> > > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > > >     void
> putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > > >     void
> > > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > > serverEndpoint),
> > > > > > what
> > > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > > >
> > > > > > > > > > > > 1004. It would be useful to document how all the new
> > > APIs are
> > > > > > > being
> > > > > > > > > used.
> > > > > > > > > > > > For example, when is
> > > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > > being set
> > > > > > > > > > > > and used? How are
> > > > > > > > > > > >
> > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > > > > > being
> > > > > > > > > used?
> > > > > > > > > > > >
> > > > > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM
> > > will
> > > > > > > > eventually
> > > > > > > > > > > > delete these segments by using RemoteStorageManager."
> > > Which
> > > > > > > replica
> > > > > > > > > does
> > > > > > > > > > > > this logic?
> > > > > > > > > > > >
> > > > > > > > > > > > 1006. "If there are any failures in removing remote
> log
> > > > > > segments
> > > > > > > > then
> > > > > > > > > > > those
> > > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > > and user can consume the events(which contain
> > > > > > > > remote-log-segment-id)
> > > > > > > > > from
> > > > > > > > > > > > that topic and clean them up from remote storage.  "
> Not
> > > sure
> > > > > > if
> > > > > > > > it's
> > > > > > > > > > > worth
> > > > > > > > > > > > the complexity of adding another topic. Could we just
> > > retry?
> > > > > > > > > > > >
> > > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> > > existing
> > > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > > >
> > > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > > remote.log.retention.minutes,
> > > > > > > > > > > > remote.log.retention.hours: It seems that we just
> need
> > > the ms
> > > > > > > one.
> > > > > > > > > Also,
> > > > > > > > > > > > are we changing the meaning of existing config
> > > > > > log.retention.ms
> > > > > > > to
> > > > > > > > > mean
> > > > > > > > > > > > the
> > > > > > > > > > > > local retention? For backward compatibility, it's
> better
> > > to not
> > > > > > > > > change
> > > > > > > > > > > the
> > > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the
> topic
> > > level?
> > > > > > > > > > > >
> > > > > > > > > > > > 1009. It would be useful to list all limitations in a
> > > separate
> > > > > > > > > section:
> > > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic
> > > from
> > > > > > delete
> > > > > > > > to
> > > > > > > > > > > > compact and vice versa allowed when tiering is
> enabled?
> > > > > > > > > > > >
> > > > > > > > > > > > 1010. Thanks for performance numbers. Are those with
> > > RocksDB as
> > > > > > > the
> > > > > > > > > > > cache?
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <
> > > harsha.ch@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > >                Thats not what we said in the
> previous
> > > email.
> > > > > > > RLMM
> > > > > > > > > is
> > > > > > > > > > > > > pluggable storage and by running numbers even 1PB
> data
> > > you do
> > > > > > > not
> > > > > > > > > need
> > > > > > > > > > > > more
> > > > > > > > > > > > > than 10GB local storage.
> > > > > > > > > > > > > If in future this becomes a blocker for any users
> we
> > > can
> > > > > > > revisit
> > > > > > > > > but
> > > > > > > > > > > this
> > > > > > > > > > > > > does not warrant another implementation at this
> point
> > > to push
> > > > > > > the
> > > > > > > > > data
> > > > > > > > > > > to
> > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > We can ofcourse implement another RLMM that is
> > > optional for
> > > > > > > users
> > > > > > > > > to
> > > > > > > > > > > > > configure to push to remote. But that doesn't need
> to
> > > be
> > > > > > > > addressed
> > > > > > > > > in
> > > > > > > > > > > > this
> > > > > > > > > > > > > KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Harsha
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe <
> > > > > > > cmccabe@apache.org
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi Ying,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for the response.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > It sounds like you agree that storing the
> metadata
> > > in the
> > > > > > > > remote
> > > > > > > > > > > > storage
> > > > > > > > > > > > > > would be a better design overall.  Given that
> that's
> > > true,
> > > > > > is
> > > > > > > > > there
> > > > > > > > > > > any
> > > > > > > > > > > > > > reason to include the worse implementation based
> on
> > > > > > RocksDB?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Choosing a long-term metadata store is not
> something
> > > that
> > > > > > we
> > > > > > > > > should
> > > > > > > > > > > do
> > > > > > > > > > > > > > lightly.  It can take users years to migrate from
> > > metadata
> > > > > > > > store
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > other.  I also don't think it's realistic or
> > > desirable for
> > > > > > > > users
> > > > > > > > > to
> > > > > > > > > > > > write
> > > > > > > > > > > > > > their own metadata stores.  Even assuming that
> they
> > > could
> > > > > > do
> > > > > > > a
> > > > > > > > > good
> > > > > > > > > > > job
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > this, it would create huge fragmentation in the
> Kafka
> > > > > > > > ecosystem.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng wrote:
> > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Satish and I are still discussing some details
> > > about how
> > > > > > to
> > > > > > > > > handle
> > > > > > > > > > > > > > > transactions / producer ids. Satish is going to
> > > make some
> > > > > > > > minor
> > > > > > > > > > > > changes
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > RLMM API and other parts. Other than that, we
> have
> > > > > > finished
> > > > > > > > > > > updating
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I agree with Colin that the current design of
> using
> > > > > > rocksDB
> > > > > > > > is
> > > > > > > > > not
> > > > > > > > > > > > > > > optimal. But this design is simple and should
> work
> > > for
> > > > > > > almost
> > > > > > > > > all
> > > > > > > > > > > the
> > > > > > > > > > > > > > > existing Kafka users. RLMM is a plugin. Users
> can
> > > replace
> > > > > > > > > rocksDB
> > > > > > > > > > > > with
> > > > > > > > > > > > > > > their own RLMM implementation, if needed. So, I
> > > think we
> > > > > > > can
> > > > > > > > > keep
> > > > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > > for now. What do you think?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Ying
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> > > > > > jun@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for the update. It's good to see the
> > > progress on
> > > > > > > > this.
> > > > > > > > > > > > Please
> > > > > > > > > > > > > > let us
> > > > > > > > > > > > > > > > know when you are done updating the KIP wiki.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng
> > > > > > > > > > > <yingz@uber.com.invalid
> > > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Satish and I have added more design
> details in
> > > the
> > > > > > KIP,
> > > > > > > > > > > including
> > > > > > > > > > > > > > how to
> > > > > > > > > > > > > > > > > keep consistency between replicas
> (especially
> > > when
> > > > > > > there
> > > > > > > > is
> > > > > > > > > > > > > > leadership
> > > > > > > > > > > > > > > > > changes / log truncations) and new
> metrics. We
> > > also
> > > > > > > made
> > > > > > > > > some
> > > > > > > > > > > > other
> > > > > > > > > > > > > > minor
> > > > > > > > > > > > > > > > > changes in the doc. We will finish the KIP
> > > changes in
> > > > > > > the
> > > > > > > > > next
> > > > > > > > > > > > > > couple of
> > > > > > > > > > > > > > > > > days. We will let you know when we are
> done.
> > > Most of
> > > > > > > the
> > > > > > > > > > > changes
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > already updated to the wiki KIP. You can
> take
> > > a look.
> > > > > > > But
> > > > > > > > > it's
> > > > > > > > > > > > not
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > As for the implementation, the code is
> mostly
> > > done
> > > > > > and
> > > > > > > we
> > > > > > > > > > > already
> > > > > > > > > > > > > had
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > feature tests / system tests. I have added
> the
> > > > > > > > performance
> > > > > > > > > test
> > > > > > > > > > > > > > results
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the KIP. However the recent design changes
> > > (e.g.
> > > > > > leader
> > > > > > > > > epoch
> > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > management / log truncation / some of the
> new
> > > > > > metrics)
> > > > > > > > > have not
> > > > > > > > > > > > > been
> > > > > > > > > > > > > > > > > implemented yet. It will take about 2 weeks
> > > for us to
> > > > > > > > > implement
> > > > > > > > > > > > > > after you
> > > > > > > > > > > > > > > > > review and agree with those design changes.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun Rao <
> > > > > > > jun@confluent.io
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Any new updates on the KIP? This feature
> is
> > > one of
> > > > > > > the
> > > > > > > > > most
> > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > most requested features in Apache Kafka
> > > right now.
> > > > > > It
> > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > > helpful
> > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > we can make sustained progress on this.
> > > Could you
> > > > > > > share
> > > > > > > > > how
> > > > > > > > > > > far
> > > > > > > > > > > > > > along
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > the design/implementation right now? Is
> there
> > > > > > > anything
> > > > > > > > > that
> > > > > > > > > > > > other
> > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > As for "transactional support" and
> "follower
> > > > > > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > > > > > further comments from me as long as the
> > > producer
> > > > > > > state
> > > > > > > > > and
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > be restored properly from the object
> store
> > > when
> > > > > > > needed.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM Satish
> > > Duggana <
> > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > We did not want to add many
> implementation
> > > > > > details
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > But we
> > > > > > > > > > > > > > > > > > > decided to add them in the KIP as
> appendix
> > > or
> > > > > > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > > > > > follower fetch protocol) to describe
> the
> > > flow
> > > > > > with
> > > > > > > > the
> > > > > > > > > main
> > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > That will answer most of the queries. I
> > > will
> > > > > > update
> > > > > > > > on
> > > > > > > > > this
> > > > > > > > > > > > > mail
> > > > > > > > > > > > > > > > > > > thread when the respective sections are
> > > updated.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM
> Alexandre
> > > Dupriez
> > > > > > > > > > > > > > > > > > > <al...@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > A couple of questions specific to the
> > > section
> > > > > > > > > "Follower
> > > > > > > > > > > > > > > > > > > > Requests/Replication", pages 16:17
> in the
> > > > > > design
> > > > > > > > > document
> > > > > > > > > > > > > [1].
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900. It is mentioned that followers
> fetch
> > > > > > > auxiliary
> > > > > > > > > > > states
> > > > > > > > > > > > > > from the
> > > > > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900.a Does the consistency model of
> the
> > > > > > external
> > > > > > > > > storage
> > > > > > > > > > > > > > impacts
> > > > > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > > > > > of leader epochs and other auxiliary
> > > data?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900.b What are the benefits of using
> a
> > > > > > mechanism
> > > > > > > to
> > > > > > > > > store
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > > > > the leader epochs which is different
> > > from other
> > > > > > > > > metadata
> > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > tiered segments? What are the
> benefits of
> > > > > > > > retrieving
> > > > > > > > > this
> > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > on-demand from the follower rather
> than
> > > relying
> > > > > > > on
> > > > > > > > > > > > > propagation
> > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > topic __remote_log_metadata? What
> are the
> > > > > > > > advantages
> > > > > > > > > over
> > > > > > > > > > > > > > using a
> > > > > > > > > > > > > > > > > > > > dedicated control structure (e.g. a
> new
> > > record
> > > > > > > > type)
> > > > > > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > > > > > this topic? Since in the document,
> > > different
> > > > > > > > control
> > > > > > > > > > > paths
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > operating in the system, how are the
> > > metadata
> > > > > > > > stored
> > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > __remote_log_metadata [which also
> > > include the
> > > > > > > epoch
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > which offloaded a segment] and the
> remote
> > > > > > > auxiliary
> > > > > > > > > > > states,
> > > > > > > > > > > > > > kept in
> > > > > > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900.c A follower can encounter an
> > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > > > > > this in response to a Fetch or
> > > > > > > OffsetForLeaderEpoch
> > > > > > > > > > > > request?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900.d What happens if, after a
> follower
> > > > > > > encountered
> > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE
> response,
> > > its
> > > > > > > > > attempts to
> > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > leader epochs fail (for instance,
> > > because the
> > > > > > > > remote
> > > > > > > > > > > > storage
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > temporarily unavailable)? Does the
> > > follower
> > > > > > > > > fallbacks to
> > > > > > > > > > > a
> > > > > > > > > > > > > mode
> > > > > > > > > > > > > > > > where
> > > > > > > > > > > > > > > > > > > > it ignores tiered segments, and
> applies
> > > > > > > truncation
> > > > > > > > > using
> > > > > > > > > > > > only
> > > > > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > > available information? What happens
> when
> > > access
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > is restored? How is the replica
> lineage
> > > > > > inferred
> > > > > > > by
> > > > > > > > > the
> > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > epochs reconciled with the follower's
> > > replica
> > > > > > > > > lineage,
> > > > > > > > > > > > which
> > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > > > evolved? Does the follower remember
> > > fetching
> > > > > > > > > auxiliary
> > > > > > > > > > > > states
> > > > > > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > > > > > in the past and attempt
> reconciliation?
> > > Is
> > > > > > there
> > > > > > > a
> > > > > > > > > plan
> > > > > > > > > > > to
> > > > > > > > > > > > > > offer
> > > > > > > > > > > > > > > > > > > > different strategies in this
> scenario,
> > > > > > > configurable
> > > > > > > > > via
> > > > > > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900.e Is the leader epoch cache
> > > offloaded with
> > > > > > > > every
> > > > > > > > > > > > segment?
> > > > > > > > > > > > > > Or
> > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > a new checkpoint is detected? If that
> > > > > > information
> > > > > > > > is
> > > > > > > > > not
> > > > > > > > > > > > > always
> > > > > > > > > > > > > > > > > > > > offloaded to avoid duplicating data,
> how
> > > does
> > > > > > the
> > > > > > > > > remote
> > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > satisfy the request to retrieve it?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900.f Since the leader epoch cache
> > > covers the
> > > > > > > > entire
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > > > > > what happens if, after a leader epoch
> > > cache
> > > > > > file
> > > > > > > is
> > > > > > > > > > > > offloaded
> > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > given segment, the local epoch cache
> is
> > > > > > truncated
> > > > > > > > > [not
> > > > > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > a range of offset included in tiered
> > > segments]?
> > > > > > > How
> > > > > > > > > are
> > > > > > > > > > > > > remote
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > local leader epoch caches kept
> > > consistent?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900.g Consumer can also use leader
> > > epochs (e.g.
> > > > > > > to
> > > > > > > > > enable
> > > > > > > > > > > > > > fencing
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > protect against stale leaders). What
> > > > > > differences
> > > > > > > > > would
> > > > > > > > > > > > there
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > between consumer and follower
> fetches?
> > > > > > > Especially,
> > > > > > > > > would
> > > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > > > > also fetch leader epoch information
> from
> > > the
> > > > > > > remote
> > > > > > > > > > > > storage?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 900.h Assume a newly elected leader
> of a
> > > > > > > > > topic-partition
> > > > > > > > > > > > > > detects
> > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > recent segments are available in the
> > > external
> > > > > > > > > storage,
> > > > > > > > > > > with
> > > > > > > > > > > > > > epochs
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > its local epoch. Does it ignore these
> > > segments
> > > > > > > and
> > > > > > > > > their
> > > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > > epoch-to-offset vectors? Or try to
> > > reconstruct
> > > > > > > its
> > > > > > > > > local
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > lineage based on the data remotely
> > > available?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > >
> https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55, Satish
> > > Duggana <
> > > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > > > Please let us know if you have any
> > > comments
> > > > > > on
> > > > > > > > > > > > > "transactional
> > > > > > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > > > > > and "follower requests/replication"
> > > mentioned
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > wiki.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25 PM
> Satish
> > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks Jun for your comments.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >100. It would be useful to
> provide
> > > more
> > > > > > > > details
> > > > > > > > > on
> > > > > > > > > > > how
> > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard
> to
> > > really
> > > > > > > > assess
> > > > > > > > > > > > whether
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > > > apis are sufficient/redundant. A few
> > > examples
> > > > > > > below.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > We will update the wiki and let
> you
> > > know.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >100.1 deleteRecords seems to
> only
> > > advance
> > > > > > > the
> > > > > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > Log. How does that trigger the
> deletion of
> > > remote
> > > > > > > log
> > > > > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > RLMTask for leader partition
> > > periodically
> > > > > > > > checks
> > > > > > > > > > > > whether
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > remote log segments earlier to
> > > > > > logStartOffset
> > > > > > > > > and the
> > > > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > > > > remote log segment metadata and
> data
> > > are
> > > > > > > > deleted
> > > > > > > > > by
> > > > > > > > > > > > using
> > > > > > > > > > > > > > RLMM
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >100.2 stopReplica with deletion
> is
> > > used
> > > > > > in 2
> > > > > > > > > cases
> > > > > > > > > > > (a)
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > reassignment; (b) topic deletion. We
> only
> > > want to
> > > > > > > > > delete
> > > > > > > > > > > the
> > > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > > metadata in the second case. Also, in
> the
> > > second
> > > > > > > > case,
> > > > > > > > > who
> > > > > > >
>

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

Posted by Satish Duggana <sa...@gmail.com>.
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 <ju...@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 <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >
> > Satish.
> >
> >
> > On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> > >
> > >
> > > Satish.
> > >
> > >
> > > On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng <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 <ju...@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 <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
> > > > > > unclean-leader-election.
> > > > > > 2) Since the user set local retention time (or local retention
> > bytes), I
> > > > > > think we are expected to
> > > > > > keep that much local data when possible (avoid truncating all the
> > local
> > > > > > data). But, as you said,
> > > > > > unclean leader elections are very rare, this may not be a big
> > problem.
> > > > > >
> > > > > > The current design uses the leader broker as source-of-truth. This
> > is
> > > > > > consistent with the
> > > > > > existing Kafka behavior.
> > > > > >
> > > > > > By using remote storage as the source-of-truth, the follower logic
> > can
> > > > > be a
> > > > > > little simpler,
> > > > > > but the leader logic is going to be more complex. Overall, I don't
> > see
> > > > > > there many benefits
> > > > > > of using remote storage as the source-of-truth.
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > > Hi, Satish,
> > > > > > >
> > > > > > > Thanks for the reply.
> > > > > > >
> > > > > > > 1001. In your example, I was thinking that you could just
> > download the
> > > > > > > latest leader epoch from the object store. After that you know
> > the
> > > > > leader
> > > > > > > should end with offset 1100. The leader will delete all its
> > local data
> > > > > > > before offset 1000 and start accepting new messages at offset
> > 1100.
> > > > > > > Consumer requests for messages before offset 1100 will be served
> > from
> > > > > the
> > > > > > > object store. The benefit with this approach is that it's
> > simpler to
> > > > > > reason
> > > > > > > about who is the source of truth. The downside is slightly
> > increased
> > > > > > > unavailability window during unclean leader election. Since
> > unclean
> > > > > > leader
> > > > > > > elections are rare, I am not sure if this is a big concern.
> > > > > > >
> > > > > > > 1008. Yes, I think introducing sth like local.retention.ms
> > seems more
> > > > > > > consistent.
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > > > > satish.duggana@gmail.com
> > > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > HI Jun,
> > > > > > > > Thanks for your comments. We put our inline replies below.
> > > > > > > >
> > > > > > > > 1001. I was thinking that you could just use the tiered
> > metadata to
> > > > > do
> > > > > > > the
> > > > > > > > reconciliation. The tiered metadata contains offset ranges and
> > epoch
> > > > > > > > history. Those should be enough for reconciliation purposes.
> > > > > > > >
> > > > > > > > If we use remote storage as the source-of-truth during
> > > > > > > > unclean-leader-election, it's possible that after
> > reconciliation the
> > > > > > > > remote storage will have more recent data than the new
> > leader's local
> > > > > > > > storage. For example, the new leader's latest message is
> > offset 1000,
> > > > > > > > while the remote storage has message 1100. In such a case, the
> > new
> > > > > > > > leader will have to download the messages from 1001 to 1100,
> > before
> > > > > > > > accepting new messages from producers. Otherwise, there would
> > be a
> > > > > gap
> > > > > > > > in the local data between 1000 and 1101.
> > > > > > > >
> > > > > > > > Moreover, with the current design, leader epoch history is
> > stored in
> > > > > > > > remote storage, rather than the metadata topic. We did consider
> > > > > saving
> > > > > > > > epoch history in remote segment metadata. But the concern is
> > that
> > > > > > > > there is currently no limit for the epoch history size.
> > > > > Theoretically,
> > > > > > > > if a user has a very long remote retention time and there are
> > very
> > > > > > > > frequent leadership changes, the leader epoch history can
> > become too
> > > > > > > > long to fit into a regular Kafka message.
> > > > > > > >
> > > > > > > >
> > > > > > > > 1003.3 Having just a serverEndpoint string is probably not
> > enough.
> > > > > > > > Connecting to a Kafka cluster may need various security
> > credentials.
> > > > > We
> > > > > > > can
> > > > > > > > make RLMM configurable and pass in the properties through the
> > > > > > configure()
> > > > > > > > method. Ditto for RSM.
> > > > > > > >
> > > > > > > > RLMM and  RSM are already configurable and they take
> > properties which
> > > > > > > > start with "remote.log.metadata." and "remote.log.storage."
> > > > > > > > respectively and a few others. We have listener-name as the
> > config
> > > > > for
> > > > > > > > RLMM and other properties(like security) can be sent as you
> > > > > suggested.
> > > > > > > > We will update the KIP with the details.
> > > > > > > >
> > > > > > > >
> > > > > > > > 1008.1 We started with log.retention.hours and
> > log.retention.minutes,
> > > > > > and
> > > > > > > > added log.retention.ms later. If we are adding a new
> > configuration,
> > > > > ms
> > > > > > > > level config alone is enough and is simpler. We can build
> > tools to
> > > > > make
> > > > > > > the
> > > > > > > > configuration at different granularities easier. The
> > definition of
> > > > > > > > log.retention.ms is "The number of milliseconds to keep a log
> > file
> > > > > > > before
> > > > > > > > deleting it". The deletion is independent of whether tiering is
> > > > > enabled
> > > > > > > or
> > > > > > > > not. If this changes to just the local portion of the data, we
> > are
> > > > > > > changing
> > > > > > > > the meaning of an existing configuration.
> > > > > > > >
> > > > > > > > We are fine with either way. We can go with log.retention.xxxx
> > as the
> > > > > > > > effective log retention instead of local log retention. With
> > this
> > > > > > > > convention, we need to introduce  local.log.retention instead
> > of
> > > > > > > > remote.log.retention.ms that we proposed. If log.retention.ms
> > as -1
> > > > > > > > then remote retention is also considered as unlimited but user
> > should
> > > > > > > > be able to set the local.retention.ms.
> > > > > > > > So, we need to introduce local.log.retention.ms and
> > > > > > > > local.log.retention.bytes which should  always  be <=
> > > > > > > > log.retention.ms/bytes respectively.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > > > >
> > > > > > > > > Hi, Satish,
> > > > > > > > >
> > > > > > > > > Thanks for the reply. A few quick comments below.
> > > > > > > > >
> > > > > > > > > 1001. I was thinking that you could just use the tiered
> > metadata to
> > > > > > do
> > > > > > > > the
> > > > > > > > > reconciliation. The tiered metadata contains offset ranges
> > and
> > > > > epoch
> > > > > > > > > history. Those should be enough for reconciliation purposes.
> > > > > > > > >
> > > > > > > > > 1003.3 Having just a serverEndpoint string is probably not
> > enough.
> > > > > > > > > Connecting to a Kafka cluster may need various security
> > > > > credentials.
> > > > > > We
> > > > > > > > can
> > > > > > > > > make RLMM configurable and pass in the properties through the
> > > > > > > configure()
> > > > > > > > > method. Ditto for RSM.
> > > > > > > > >
> > > > > > > > > 1008.1 We started with log.retention.hours and
> > > > > log.retention.minutes,
> > > > > > > and
> > > > > > > > > added log.retention.ms later. If we are adding a new
> > > > > configuration,
> > > > > > ms
> > > > > > > > > level config alone is enough and is simpler. We can build
> > tools to
> > > > > > make
> > > > > > > > the
> > > > > > > > > configuration at different granularities easier. The
> > definition of
> > > > > > > > > log.retention.ms is "The number of milliseconds to keep a
> > log file
> > > > > > > > before
> > > > > > > > > deleting it". The deletion is independent of whether tiering
> > is
> > > > > > enabled
> > > > > > > > or
> > > > > > > > > not. If this changes to just the local portion of the data,
> > we are
> > > > > > > > changing
> > > > > > > > > the meaning of an existing configuration.
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Jun,
> > > > > > > > > >
> > > > > > > > > > Thank you for the comments! Ying, Harsha and I discussed
> > and put
> > > > > > our
> > > > > > > > > > comments below.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > > > > elections.
> > > > > > > > This
> > > > > > > > > > is very useful, but I am wondering if this is the best
> > approach.
> > > > > My
> > > > > > > > > > understanding of the proposed approach is to allow the new
> > > > > > (unclean)
> > > > > > > > leader
> > > > > > > > > > to take new messages immediately. While this increases
> > > > > > availability,
> > > > > > > it
> > > > > > > > > > creates the problem that there could be multiple
> > conflicting
> > > > > > segments
> > > > > > > > in
> > > > > > > > > > the remote store for the same offset range. This seems to
> > make it
> > > > > > > > harder
> > > > > > > > > > for RLMM to determine which archived log segments contain
> > the
> > > > > > correct
> > > > > > > > data.
> > > > > > > > > > For example, an archived log segment could at one time be
> > the
> > > > > > correct
> > > > > > > > data,
> > > > > > > > > > but be changed to incorrect data after an unclean leader
> > > > > election.
> > > > > > An
> > > > > > > > > > alternative approach is to let the unclean leader use the
> > > > > archived
> > > > > > > > data as
> > > > > > > > > > the source of truth. So, when the new (unclean) leader
> > takes
> > > > > over,
> > > > > > it
> > > > > > > > first
> > > > > > > > > > reconciles the local data based on the archived data before
> > > > > taking
> > > > > > > new
> > > > > > > > > > messages. This makes the job of RLMM a bit easier since all
> > > > > > archived
> > > > > > > > data
> > > > > > > > > > are considered correct. This increases availability a bit.
> > > > > However,
> > > > > > > > since
> > > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > > >
> > > > > > > > > > Firstly, We don't want to assume the remote storage is more
> > > > > > reliable
> > > > > > > > than
> > > > > > > > > > Kafka. Kafka unclean leader election usually happens when
> > there
> > > > > is
> > > > > > a
> > > > > > > > large
> > > > > > > > > > scale outage that impacts multiple racks (or even multiple
> > > > > > > availability
> > > > > > > > > > zones). In such a case, the remote storage may be
> > unavailable or
> > > > > > > > unstable.
> > > > > > > > > > Pulling a large amount of data from the remote storage to
> > > > > reconcile
> > > > > > > the
> > > > > > > > > > local data may also exacerbate the outage. With the current
> > > > > design,
> > > > > > > > the new
> > > > > > > > > > leader can start working even when the remote storage is
> > > > > > temporarily
> > > > > > > > > > unavailable.
> > > > > > > > > >
> > > > > > > > > > Secondly, it is not easier to implement the reconciling
> > logic at
> > > > > > the
> > > > > > > > leader
> > > > > > > > > > side. It can take a long time for the new leader to
> > download the
> > > > > > > remote
> > > > > > > > > > data and rebuild local producer id / leader epoch
> > information.
> > > > > > During
> > > > > > > > this
> > > > > > > > > > period, the leader cannot accept any requests from the
> > clients
> > > > > and
> > > > > > > > > > followers. We have to introduce a new state for the
> > leader, and a
> > > > > > new
> > > > > > > > error
> > > > > > > > > > code to let the clients / followers know what is happening.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > RemoteStorageManager.
> > > > > > > > We
> > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all other
> > > > > methods,
> > > > > > > we
> > > > > > > > pass
> > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > >
> > > > > > > > > > Nice catch, we can have the RemoteLogSegmentMetadata for
> > > > > > > copyLogSegment
> > > > > > > > > > too.
> > > > > > > > > >
> > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata inclusive
> > or
> > > > > > > exclusive?
> > > > > > > > > >
> > > > > > > > > > It is inclusive.
> > > > > > > > > >
> > > > > > > > > > 1002.3 It seems that we need an api to get the leaderEpoch
> > > > > history
> > > > > > > for
> > > > > > > > a
> > > > > > > > > > partition.
> > > > > > > > > >
> > > > > > > > > > Yes, updated the KIP with the new method.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 1002.4 Could you define the type of
> > RemoteLogSegmentContext?
> > > > > > > > > >
> > > > > > > > > > This is removed in the latest code and it is not needed.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > >
> > > > > > > > > > 1003.1 I am not sure why we need both of the following
> > methods
> > > > > > > > > > in RemoteLogMetadataManager. Could we combine them into
> > one that
> > > > > > > takes
> > > > > > > > in
> > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > >     RemoteLogSegmentId getRemoteLogSegmentId(TopicPartition
> > > > > > > > topicPartition,
> > > > > > > > > > long offset) throws IOException;
> > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > >
> > > > > > > > > > Good point, these can be merged for now. I guess we needed
> > them
> > > > > in
> > > > > > > > earlier
> > > > > > > > > > version of the implementation but it is not needed now.
> > > > > > > > > >
> > > > > > > > > > 1003.2 There seems to be some inconsistencies in the
> > methods
> > > > > > below. I
> > > > > > > > am
> > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the
> > other
> > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > >     void deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > >
> > > > > > > > > > RLMM stores RemoteLogSegmentMetadata which is identified by
> > > > > > > > > > RemoteLogsSegmentId. So, when it is added it takes
> > > > > > > > > > RemoteLogSegmentMetadata. `delete` operation needs only
> > > > > > > > RemoteLogsSegmentId
> > > > > > > > > > as RemoteLogSegmentMetadata can be identified with
> > > > > > > RemoteLogsSegmentId.
> > > > > > > > > >
> > > > > > > > > > 1003.3 In void onServerStarted(final String
> > serverEndpoint), what
> > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > >
> > > > > > > > > > This can be used by RLMM implementation to connect to the
> > local
> > > > > > Kafka
> > > > > > > > > > cluster. Incase of default  implementation, it is used in
> > > > > > > initializing
> > > > > > > > > > kafka clients connecting to the local cluster.
> > > > > > > > > >
> > > > > > > > > > 1004. It would be useful to document how all the new APIs
> > are
> > > > > being
> > > > > > > > used.
> > > > > > > > > > For example, when is
> > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > being
> > > > > > > > set
> > > > > > > > > > and used? How are
> > > > > > > > > >
> > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > > > > > used?
> > > > > > > > > >
> > > > > > > > > > RLMM APIs are going through the changes and they should be
> > ready
> > > > > > in a
> > > > > > > > few
> > > > > > > > > > days. I will update the KIP and the mail  thread once they
> > are
> > > > > > ready.
> > > > > > > > > >
> > > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM will
> > > > > > eventually
> > > > > > > > > > delete these segments by using RemoteStorageManager." Which
> > > > > replica
> > > > > > > > does
> > > > > > > > > > this logic?
> > > > > > > > > >
> > > > > > > > > > This is a good point. When a topic is deleted, it will not
> > have
> > > > > any
> > > > > > > > > > leader/followers to do the cleanup. We will have a cleaner
> > agent
> > > > > > on a
> > > > > > > > > > single broker in the cluster to do this cleanup, we plan
> > to add
> > > > > > that
> > > > > > > in
> > > > > > > > > > controller broker.
> > > > > > > > > >
> > > > > > > > > > 1006. "If there are any failures in removing remote log
> > segments
> > > > > > then
> > > > > > > > those
> > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > and user can consume the events(which contain
> > > > > > remote-log-segment-id)
> > > > > > > > from
> > > > > > > > > > that topic and clean them up from remote storage.  " Not
> > sure if
> > > > > > it's
> > > > > > > > worth
> > > > > > > > > > the complexity of adding another topic. Could we just
> > retry?
> > > > > > > > > >
> > > > > > > > > > Sure, we can keep this simpler for now by logging an error
> > after
> > > > > > > > retries.
> > > > > > > > > > We can give users a better way to process this in future.
> > Oneway
> > > > > > can
> > > > > > > > be a
> > > > > > > > > > dead letter topic which can be configured by the user.
> > > > > > > > > >
> > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> > existing
> > > > > > > > > > fetchPurgatory?
> > > > > > > > > >
> > > > > > > > > > We have 2 types of delayed operations waiting for 2
> > different
> > > > > > events.
> > > > > > > > > > DelayedFetch waits for new messages from producers.
> > > > > > > DelayedRemoteFetch
> > > > > > > > > > waits for the remote-storage-read-task to finish. When
> > either of
> > > > > > the
> > > > > > > 2
> > > > > > > > > > events happens, we only want to notify one type of the
> > delayed
> > > > > > > > operations.
> > > > > > > > > > It would be inefficient to put 2 types of delayed
> > operations in
> > > > > one
> > > > > > > > > > purgatory, as the tryComplete() methods of the delayed
> > operations
> > > > > > can
> > > > > > > > be
> > > > > > > > > > triggered by irrelevant events.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 1008. Configurations:
> > > > > > > > > > 1008.1 remote.log.retention.ms,
> > remote.log.retention.minutes,
> > > > > > > > > > remote.log.retention.hours: It seems that we just need the
> > ms
> > > > > one.
> > > > > > > > Also,
> > > > > > > > > > are we changing the meaning of existing config
> > log.retention.ms
> > > > > to
> > > > > > > > mean
> > > > > > > > > > the
> > > > > > > > > > local retention? For backward compatibility, it's better
> > to not
> > > > > > > change
> > > > > > > > the
> > > > > > > > > > meaning of existing configurations.
> > > > > > > > > >
> > > > > > > > > > We agree that we only need remote.log.retention.ms. But,
> > the
> > > > > > > existing
> > > > > > > > > > Kafka
> > > > > > > > > > configuration
> > > > > > > > > > has 3 properties (log.retention.ms, log.retention.minutes,
> > > > > > > > > > log.retention.hours). We just
> > > > > > > > > > want to keep consistent with the existing properties.
> > > > > > > > > > Existing log.retention.xxxx config is about log retention
> > in
> > > > > > broker’s
> > > > > > > > > > storage which is local. It should be easy for users to
> > configure
> > > > > > > > partition
> > > > > > > > > > storage with local retention and remote retention based on
> > their
> > > > > > > usage.
> > > > > > > > > >
> > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic
> > level?
> > > > > > > > > >
> > > > > > > > > > We can introduce topic level config for the same remote.log
> > > > > > settings.
> > > > > > > > User
> > > > > > > > > > can set the desired config while creating the topic.
> > > > > > > > > > remote.log.storage.enable property is not allowed to be
> > updated
> > > > > > after
> > > > > > > > the
> > > > > > > > > > topic is created. Other remote.log.* properties can be
> > modified.
> > > > > We
> > > > > > > > will
> > > > > > > > > > support flipping remote.log.storage.enable in next
> > versions.
> > > > > > > > > >
> > > > > > > > > > 1009. It would be useful to list all limitations in a
> > separate
> > > > > > > section:
> > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic from
> > delete
> > > > > > to
> > > > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > > > >
> > > > > > > > > > +1 to have limitations in a separate section. We will
> > update the
> > > > > > KIP
> > > > > > > > with
> > > > > > > > > > that.
> > > > > > > > > > Topic  created with effective value for remote.log.enabled
> > as
> > > > > true,
> > > > > > > > can not
> > > > > > > > > > change its retention policy from delete to compact.
> > > > > > > > > >
> > > > > > > > > > 1010. Thanks for performance numbers. Are those with
> > RocksDB as
> > > > > the
> > > > > > > > cache?
> > > > > > > > > >
> > > > > > > > > > No, We have not yet added RocksDB support. This is based on
> > > > > > in-memory
> > > > > > > > map
> > > > > > > > > > representation. We will add that support and update this
> > thread
> > > > > > after
> > > > > > > > > > updating the KIP with the numbers.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Satish.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <ju...@confluent.io>
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > > > >
> > > > > > > > > > > 1000. Regarding Colin's question on querying the metadata
> > > > > > directly
> > > > > > > > in the
> > > > > > > > > > > remote block store. One issue is that not all block
> > stores
> > > > > offer
> > > > > > > the
> > > > > > > > > > needed
> > > > > > > > > > > api to query the metadata. For example, S3 only offers
> > an api
> > > > > to
> > > > > > > list
> > > > > > > > > > > objects under a prefix and this api has the eventual
> > > > > consistency
> > > > > > > > > > semantic.
> > > > > > > > > > >
> > > > > > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > > > > > elections.
> > > > > > > > This
> > > > > > > > > > > is very useful, but I am wondering if this is the best
> > > > > approach.
> > > > > > My
> > > > > > > > > > > understanding of the proposed approach is to allow the
> > new
> > > > > > > (unclean)
> > > > > > > > > > leader
> > > > > > > > > > > to take new messages immediately. While this increases
> > > > > > > availability,
> > > > > > > > it
> > > > > > > > > > > creates the problem that there could be multiple
> > conflicting
> > > > > > > > segments in
> > > > > > > > > > > the remote store for the same offset range. This seems
> > to make
> > > > > it
> > > > > > > > harder
> > > > > > > > > > > for RLMM to determine which archived log segments
> > contain the
> > > > > > > correct
> > > > > > > > > > data.
> > > > > > > > > > > For example, an archived log segment could at one time
> > be the
> > > > > > > correct
> > > > > > > > > > data,
> > > > > > > > > > > but be changed to incorrect data after an unclean leader
> > > > > > election.
> > > > > > > An
> > > > > > > > > > > alternative approach is to let the unclean leader use the
> > > > > > archived
> > > > > > > > data
> > > > > > > > > > as
> > > > > > > > > > > the source of truth. So, when the new (unclean) leader
> > takes
> > > > > > over,
> > > > > > > it
> > > > > > > > > > first
> > > > > > > > > > > reconciles the local data based on the archived data
> > before
> > > > > > taking
> > > > > > > > new
> > > > > > > > > > > messages. This makes the job of RLMM a bit easier since
> > all
> > > > > > > archived
> > > > > > > > data
> > > > > > > > > > > are considered correct. This increases availability a
> > bit.
> > > > > > However,
> > > > > > > > since
> > > > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > > > >
> > > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > > RemoteStorageManager. We
> > > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all
> > other
> > > > > > methods,
> > > > > > > > we
> > > > > > > > > > pass
> > > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> > inclusive or
> > > > > > > > exclusive?
> > > > > > > > > > > 1002.3 It seems that we need an api to get the
> > leaderEpoch
> > > > > > history
> > > > > > > > for a
> > > > > > > > > > > partition.
> > > > > > > > > > > 1002.4 Could you define the type of
> > RemoteLogSegmentContext?
> > > > > > > > > > >
> > > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > > 1003.1 I am not sure why we need both of the following
> > methods
> > > > > > > > > > > in RemoteLogMetadataManager. Could we combine them into
> > one
> > > > > that
> > > > > > > > takes in
> > > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > > >     RemoteLogSegmentId
> > getRemoteLogSegmentId(TopicPartition
> > > > > > > > > > topicPartition,
> > > > > > > > > > > long offset) throws IOException;
> > > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > 1003.2 There seems to be some inconsistencies in the
> > methods
> > > > > > below.
> > > > > > > > I am
> > > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the
> > other
> > > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > > >     void
> > deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > > 1003.3 In void onServerStarted(final String
> > serverEndpoint),
> > > > > what
> > > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > > >
> > > > > > > > > > > 1004. It would be useful to document how all the new
> > APIs are
> > > > > > being
> > > > > > > > used.
> > > > > > > > > > > For example, when is
> > RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > > being set
> > > > > > > > > > > and used? How are
> > > > > > > > > > >
> > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > > > > being
> > > > > > > > used?
> > > > > > > > > > >
> > > > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM
> > will
> > > > > > > eventually
> > > > > > > > > > > delete these segments by using RemoteStorageManager."
> > Which
> > > > > > replica
> > > > > > > > does
> > > > > > > > > > > this logic?
> > > > > > > > > > >
> > > > > > > > > > > 1006. "If there are any failures in removing remote log
> > > > > segments
> > > > > > > then
> > > > > > > > > > those
> > > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > > and user can consume the events(which contain
> > > > > > > remote-log-segment-id)
> > > > > > > > from
> > > > > > > > > > > that topic and clean them up from remote storage.  " Not
> > sure
> > > > > if
> > > > > > > it's
> > > > > > > > > > worth
> > > > > > > > > > > the complexity of adding another topic. Could we just
> > retry?
> > > > > > > > > > >
> > > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> > existing
> > > > > > > > > > > fetchPurgatory?
> > > > > > > > > > >
> > > > > > > > > > > 1008. Configurations:
> > > > > > > > > > > 1008.1 remote.log.retention.ms,
> > remote.log.retention.minutes,
> > > > > > > > > > > remote.log.retention.hours: It seems that we just need
> > the ms
> > > > > > one.
> > > > > > > > Also,
> > > > > > > > > > > are we changing the meaning of existing config
> > > > > log.retention.ms
> > > > > > to
> > > > > > > > mean
> > > > > > > > > > > the
> > > > > > > > > > > local retention? For backward compatibility, it's better
> > to not
> > > > > > > > change
> > > > > > > > > > the
> > > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic
> > level?
> > > > > > > > > > >
> > > > > > > > > > > 1009. It would be useful to list all limitations in a
> > separate
> > > > > > > > section:
> > > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic
> > from
> > > > > delete
> > > > > > > to
> > > > > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > > > > >
> > > > > > > > > > > 1010. Thanks for performance numbers. Are those with
> > RocksDB as
> > > > > > the
> > > > > > > > > > cache?
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <
> > harsha.ch@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > >                Thats not what we said in the previous
> > email.
> > > > > > RLMM
> > > > > > > > is
> > > > > > > > > > > > pluggable storage and by running numbers even 1PB data
> > you do
> > > > > > not
> > > > > > > > need
> > > > > > > > > > > more
> > > > > > > > > > > > than 10GB local storage.
> > > > > > > > > > > > If in future this becomes a blocker for any users we
> > can
> > > > > > revisit
> > > > > > > > but
> > > > > > > > > > this
> > > > > > > > > > > > does not warrant another implementation at this point
> > to push
> > > > > > the
> > > > > > > > data
> > > > > > > > > > to
> > > > > > > > > > > > remote storage.
> > > > > > > > > > > > We can ofcourse implement another RLMM that is
> > optional for
> > > > > > users
> > > > > > > > to
> > > > > > > > > > > > configure to push to remote. But that doesn't need to
> > be
> > > > > > > addressed
> > > > > > > > in
> > > > > > > > > > > this
> > > > > > > > > > > > KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Harsha
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe <
> > > > > > cmccabe@apache.org
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi Ying,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the response.
> > > > > > > > > > > > >
> > > > > > > > > > > > > It sounds like you agree that storing the metadata
> > in the
> > > > > > > remote
> > > > > > > > > > > storage
> > > > > > > > > > > > > would be a better design overall.  Given that that's
> > true,
> > > > > is
> > > > > > > > there
> > > > > > > > > > any
> > > > > > > > > > > > > reason to include the worse implementation based on
> > > > > RocksDB?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Choosing a long-term metadata store is not something
> > that
> > > > > we
> > > > > > > > should
> > > > > > > > > > do
> > > > > > > > > > > > > lightly.  It can take users years to migrate from
> > metadata
> > > > > > > store
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > other.  I also don't think it's realistic or
> > desirable for
> > > > > > > users
> > > > > > > > to
> > > > > > > > > > > write
> > > > > > > > > > > > > their own metadata stores.  Even assuming that they
> > could
> > > > > do
> > > > > > a
> > > > > > > > good
> > > > > > > > > > job
> > > > > > > > > > > > at
> > > > > > > > > > > > > this, it would create huge fragmentation in the Kafka
> > > > > > > ecosystem.
> > > > > > > > > > > > >
> > > > > > > > > > > > > best,
> > > > > > > > > > > > > Colin
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng wrote:
> > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Satish and I are still discussing some details
> > about how
> > > > > to
> > > > > > > > handle
> > > > > > > > > > > > > > transactions / producer ids. Satish is going to
> > make some
> > > > > > > minor
> > > > > > > > > > > changes
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > RLMM API and other parts. Other than that, we have
> > > > > finished
> > > > > > > > > > updating
> > > > > > > > > > > > the
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I agree with Colin that the current design of using
> > > > > rocksDB
> > > > > > > is
> > > > > > > > not
> > > > > > > > > > > > > > optimal. But this design is simple and should work
> > for
> > > > > > almost
> > > > > > > > all
> > > > > > > > > > the
> > > > > > > > > > > > > > existing Kafka users. RLMM is a plugin. Users can
> > replace
> > > > > > > > rocksDB
> > > > > > > > > > > with
> > > > > > > > > > > > > > their own RLMM implementation, if needed. So, I
> > think we
> > > > > > can
> > > > > > > > keep
> > > > > > > > > > > > rocksDB
> > > > > > > > > > > > > > for now. What do you think?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Ying
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> > > > > jun@confluent.io>
> > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for the update. It's good to see the
> > progress on
> > > > > > > this.
> > > > > > > > > > > Please
> > > > > > > > > > > > > let us
> > > > > > > > > > > > > > > know when you are done updating the KIP wiki.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng
> > > > > > > > > > <yingz@uber.com.invalid
> > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Satish and I have added more design details in
> > the
> > > > > KIP,
> > > > > > > > > > including
> > > > > > > > > > > > > how to
> > > > > > > > > > > > > > > > keep consistency between replicas (especially
> > when
> > > > > > there
> > > > > > > is
> > > > > > > > > > > > > leadership
> > > > > > > > > > > > > > > > changes / log truncations) and new metrics. We
> > also
> > > > > > made
> > > > > > > > some
> > > > > > > > > > > other
> > > > > > > > > > > > > minor
> > > > > > > > > > > > > > > > changes in the doc. We will finish the KIP
> > changes in
> > > > > > the
> > > > > > > > next
> > > > > > > > > > > > > couple of
> > > > > > > > > > > > > > > > days. We will let you know when we are done.
> > Most of
> > > > > > the
> > > > > > > > > > changes
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > already updated to the wiki KIP. You can take
> > a look.
> > > > > > But
> > > > > > > > it's
> > > > > > > > > > > not
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > As for the implementation, the code is mostly
> > done
> > > > > and
> > > > > > we
> > > > > > > > > > already
> > > > > > > > > > > > had
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > feature tests / system tests. I have added the
> > > > > > > performance
> > > > > > > > test
> > > > > > > > > > > > > results
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the KIP. However the recent design changes
> > (e.g.
> > > > > leader
> > > > > > > > epoch
> > > > > > > > > > > info
> > > > > > > > > > > > > > > > management / log truncation / some of the new
> > > > > metrics)
> > > > > > > > have not
> > > > > > > > > > > > been
> > > > > > > > > > > > > > > > implemented yet. It will take about 2 weeks
> > for us to
> > > > > > > > implement
> > > > > > > > > > > > > after you
> > > > > > > > > > > > > > > > review and agree with those design changes.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun Rao <
> > > > > > jun@confluent.io
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Any new updates on the KIP? This feature is
> > one of
> > > > > > the
> > > > > > > > most
> > > > > > > > > > > > > important
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > most requested features in Apache Kafka
> > right now.
> > > > > It
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > helpful
> > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > we can make sustained progress on this.
> > Could you
> > > > > > share
> > > > > > > > how
> > > > > > > > > > far
> > > > > > > > > > > > > along
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > the design/implementation right now? Is there
> > > > > > anything
> > > > > > > > that
> > > > > > > > > > > other
> > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > As for "transactional support" and "follower
> > > > > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > > > > further comments from me as long as the
> > producer
> > > > > > state
> > > > > > > > and
> > > > > > > > > > > leader
> > > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > be restored properly from the object store
> > when
> > > > > > needed.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM Satish
> > Duggana <
> > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > We did not want to add many implementation
> > > > > details
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > KIP.
> > > > > > > > > > > > > But we
> > > > > > > > > > > > > > > > > > decided to add them in the KIP as appendix
> > or
> > > > > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > > > > follower fetch protocol) to describe the
> > flow
> > > > > with
> > > > > > > the
> > > > > > > > main
> > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > That will answer most of the queries. I
> > will
> > > > > update
> > > > > > > on
> > > > > > > > this
> > > > > > > > > > > > mail
> > > > > > > > > > > > > > > > > > thread when the respective sections are
> > updated.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM Alexandre
> > Dupriez
> > > > > > > > > > > > > > > > > > <al...@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > A couple of questions specific to the
> > section
> > > > > > > > "Follower
> > > > > > > > > > > > > > > > > > > Requests/Replication", pages 16:17 in the
> > > > > design
> > > > > > > > document
> > > > > > > > > > > > [1].
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900. It is mentioned that followers fetch
> > > > > > auxiliary
> > > > > > > > > > states
> > > > > > > > > > > > > from the
> > > > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900.a Does the consistency model of the
> > > > > external
> > > > > > > > storage
> > > > > > > > > > > > > impacts
> > > > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > > > > of leader epochs and other auxiliary
> > data?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900.b What are the benefits of using a
> > > > > mechanism
> > > > > > to
> > > > > > > > store
> > > > > > > > > > > and
> > > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > > > the leader epochs which is different
> > from other
> > > > > > > > metadata
> > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > tiered segments? What are the benefits of
> > > > > > > retrieving
> > > > > > > > this
> > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > on-demand from the follower rather than
> > relying
> > > > > > on
> > > > > > > > > > > > propagation
> > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > topic __remote_log_metadata? What are the
> > > > > > > advantages
> > > > > > > > over
> > > > > > > > > > > > > using a
> > > > > > > > > > > > > > > > > > > dedicated control structure (e.g. a new
> > record
> > > > > > > type)
> > > > > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > > > > this topic? Since in the document,
> > different
> > > > > > > control
> > > > > > > > > > paths
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > operating in the system, how are the
> > metadata
> > > > > > > stored
> > > > > > > > in
> > > > > > > > > > > > > > > > > > > __remote_log_metadata [which also
> > include the
> > > > > > epoch
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > which offloaded a segment] and the remote
> > > > > > auxiliary
> > > > > > > > > > states,
> > > > > > > > > > > > > kept in
> > > > > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900.c A follower can encounter an
> > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > > > > this in response to a Fetch or
> > > > > > OffsetForLeaderEpoch
> > > > > > > > > > > request?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900.d What happens if, after a follower
> > > > > > encountered
> > > > > > > > an
> > > > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE response,
> > its
> > > > > > > > attempts to
> > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > leader epochs fail (for instance,
> > because the
> > > > > > > remote
> > > > > > > > > > > storage
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > temporarily unavailable)? Does the
> > follower
> > > > > > > > fallbacks to
> > > > > > > > > > a
> > > > > > > > > > > > mode
> > > > > > > > > > > > > > > where
> > > > > > > > > > > > > > > > > > > it ignores tiered segments, and applies
> > > > > > truncation
> > > > > > > > using
> > > > > > > > > > > only
> > > > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > available information? What happens when
> > access
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > remote
> > > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > is restored? How is the replica lineage
> > > > > inferred
> > > > > > by
> > > > > > > > the
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > epochs reconciled with the follower's
> > replica
> > > > > > > > lineage,
> > > > > > > > > > > which
> > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > > evolved? Does the follower remember
> > fetching
> > > > > > > > auxiliary
> > > > > > > > > > > states
> > > > > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > > > > in the past and attempt reconciliation?
> > Is
> > > > > there
> > > > > > a
> > > > > > > > plan
> > > > > > > > > > to
> > > > > > > > > > > > > offer
> > > > > > > > > > > > > > > > > > > different strategies in this scenario,
> > > > > > configurable
> > > > > > > > via
> > > > > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900.e Is the leader epoch cache
> > offloaded with
> > > > > > > every
> > > > > > > > > > > segment?
> > > > > > > > > > > > > Or
> > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > a new checkpoint is detected? If that
> > > > > information
> > > > > > > is
> > > > > > > > not
> > > > > > > > > > > > always
> > > > > > > > > > > > > > > > > > > offloaded to avoid duplicating data, how
> > does
> > > > > the
> > > > > > > > remote
> > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > satisfy the request to retrieve it?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900.f Since the leader epoch cache
> > covers the
> > > > > > > entire
> > > > > > > > > > > replica
> > > > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > > > > what happens if, after a leader epoch
> > cache
> > > > > file
> > > > > > is
> > > > > > > > > > > offloaded
> > > > > > > > > > > > > with
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > given segment, the local epoch cache is
> > > > > truncated
> > > > > > > > [not
> > > > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > a range of offset included in tiered
> > segments]?
> > > > > > How
> > > > > > > > are
> > > > > > > > > > > > remote
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > local leader epoch caches kept
> > consistent?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900.g Consumer can also use leader
> > epochs (e.g.
> > > > > > to
> > > > > > > > enable
> > > > > > > > > > > > > fencing
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > protect against stale leaders). What
> > > > > differences
> > > > > > > > would
> > > > > > > > > > > there
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > between consumer and follower fetches?
> > > > > > Especially,
> > > > > > > > would
> > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > > > also fetch leader epoch information from
> > the
> > > > > > remote
> > > > > > > > > > > storage?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 900.h Assume a newly elected leader of a
> > > > > > > > topic-partition
> > > > > > > > > > > > > detects
> > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > recent segments are available in the
> > external
> > > > > > > > storage,
> > > > > > > > > > with
> > > > > > > > > > > > > epochs
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > its local epoch. Does it ignore these
> > segments
> > > > > > and
> > > > > > > > their
> > > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > > epoch-to-offset vectors? Or try to
> > reconstruct
> > > > > > its
> > > > > > > > local
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > lineage based on the data remotely
> > available?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55, Satish
> > Duggana <
> > > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > > Please let us know if you have any
> > comments
> > > > > on
> > > > > > > > > > > > "transactional
> > > > > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > > > > and "follower requests/replication"
> > mentioned
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > wiki.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25 PM Satish
> > > > > Duggana <
> > > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks Jun for your comments.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >100. It would be useful to provide
> > more
> > > > > > > details
> > > > > > > > on
> > > > > > > > > > how
> > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to
> > really
> > > > > > > assess
> > > > > > > > > > > whether
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > > apis are sufficient/redundant. A few
> > examples
> > > > > > below.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > We will update the wiki and let you
> > know.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >100.1 deleteRecords seems to only
> > advance
> > > > > > the
> > > > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > Log. How does that trigger the deletion of
> > remote
> > > > > > log
> > > > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > RLMTask for leader partition
> > periodically
> > > > > > > checks
> > > > > > > > > > > whether
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > remote log segments earlier to
> > > > > logStartOffset
> > > > > > > > and the
> > > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > > > remote log segment metadata and data
> > are
> > > > > > > deleted
> > > > > > > > by
> > > > > > > > > > > using
> > > > > > > > > > > > > RLMM
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >100.2 stopReplica with deletion is
> > used
> > > > > in 2
> > > > > > > > cases
> > > > > > > > > > (a)
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > reassignment; (b) topic deletion. We only
> > want to
> > > > > > > > delete
> > > > > > > > > > the
> > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > metadata in the second case. Also, in the
> > second
> > > > > > > case,
> > > > > > > > who
> > > > > >

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

Posted by Jun Rao <ju...@confluent.io>.
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 <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>
> Satish.
>
>
> On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
> >
> >
> > Satish.
> >
> >
> > On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng <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 <ju...@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 <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
> > > > > unclean-leader-election.
> > > > > 2) Since the user set local retention time (or local retention
> bytes), I
> > > > > think we are expected to
> > > > > keep that much local data when possible (avoid truncating all the
> local
> > > > > data). But, as you said,
> > > > > unclean leader elections are very rare, this may not be a big
> problem.
> > > > >
> > > > > The current design uses the leader broker as source-of-truth. This
> is
> > > > > consistent with the
> > > > > existing Kafka behavior.
> > > > >
> > > > > By using remote storage as the source-of-truth, the follower logic
> can
> > > > be a
> > > > > little simpler,
> > > > > but the leader logic is going to be more complex. Overall, I don't
> see
> > > > > there many benefits
> > > > > of using remote storage as the source-of-truth.
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Hi, Satish,
> > > > > >
> > > > > > Thanks for the reply.
> > > > > >
> > > > > > 1001. In your example, I was thinking that you could just
> download the
> > > > > > latest leader epoch from the object store. After that you know
> the
> > > > leader
> > > > > > should end with offset 1100. The leader will delete all its
> local data
> > > > > > before offset 1000 and start accepting new messages at offset
> 1100.
> > > > > > Consumer requests for messages before offset 1100 will be served
> from
> > > > the
> > > > > > object store. The benefit with this approach is that it's
> simpler to
> > > > > reason
> > > > > > about who is the source of truth. The downside is slightly
> increased
> > > > > > unavailability window during unclean leader election. Since
> unclean
> > > > > leader
> > > > > > elections are rare, I am not sure if this is a big concern.
> > > > > >
> > > > > > 1008. Yes, I think introducing sth like local.retention.ms
> seems more
> > > > > > consistent.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > > > satish.duggana@gmail.com
> > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > HI Jun,
> > > > > > > Thanks for your comments. We put our inline replies below.
> > > > > > >
> > > > > > > 1001. I was thinking that you could just use the tiered
> metadata to
> > > > do
> > > > > > the
> > > > > > > reconciliation. The tiered metadata contains offset ranges and
> epoch
> > > > > > > history. Those should be enough for reconciliation purposes.
> > > > > > >
> > > > > > > If we use remote storage as the source-of-truth during
> > > > > > > unclean-leader-election, it's possible that after
> reconciliation the
> > > > > > > remote storage will have more recent data than the new
> leader's local
> > > > > > > storage. For example, the new leader's latest message is
> offset 1000,
> > > > > > > while the remote storage has message 1100. In such a case, the
> new
> > > > > > > leader will have to download the messages from 1001 to 1100,
> before
> > > > > > > accepting new messages from producers. Otherwise, there would
> be a
> > > > gap
> > > > > > > in the local data between 1000 and 1101.
> > > > > > >
> > > > > > > Moreover, with the current design, leader epoch history is
> stored in
> > > > > > > remote storage, rather than the metadata topic. We did consider
> > > > saving
> > > > > > > epoch history in remote segment metadata. But the concern is
> that
> > > > > > > there is currently no limit for the epoch history size.
> > > > Theoretically,
> > > > > > > if a user has a very long remote retention time and there are
> very
> > > > > > > frequent leadership changes, the leader epoch history can
> become too
> > > > > > > long to fit into a regular Kafka message.
> > > > > > >
> > > > > > >
> > > > > > > 1003.3 Having just a serverEndpoint string is probably not
> enough.
> > > > > > > Connecting to a Kafka cluster may need various security
> credentials.
> > > > We
> > > > > > can
> > > > > > > make RLMM configurable and pass in the properties through the
> > > > > configure()
> > > > > > > method. Ditto for RSM.
> > > > > > >
> > > > > > > RLMM and  RSM are already configurable and they take
> properties which
> > > > > > > start with "remote.log.metadata." and "remote.log.storage."
> > > > > > > respectively and a few others. We have listener-name as the
> config
> > > > for
> > > > > > > RLMM and other properties(like security) can be sent as you
> > > > suggested.
> > > > > > > We will update the KIP with the details.
> > > > > > >
> > > > > > >
> > > > > > > 1008.1 We started with log.retention.hours and
> log.retention.minutes,
> > > > > and
> > > > > > > added log.retention.ms later. If we are adding a new
> configuration,
> > > > ms
> > > > > > > level config alone is enough and is simpler. We can build
> tools to
> > > > make
> > > > > > the
> > > > > > > configuration at different granularities easier. The
> definition of
> > > > > > > log.retention.ms is "The number of milliseconds to keep a log
> file
> > > > > > before
> > > > > > > deleting it". The deletion is independent of whether tiering is
> > > > enabled
> > > > > > or
> > > > > > > not. If this changes to just the local portion of the data, we
> are
> > > > > > changing
> > > > > > > the meaning of an existing configuration.
> > > > > > >
> > > > > > > We are fine with either way. We can go with log.retention.xxxx
> as the
> > > > > > > effective log retention instead of local log retention. With
> this
> > > > > > > convention, we need to introduce  local.log.retention instead
> of
> > > > > > > remote.log.retention.ms that we proposed. If log.retention.ms
> as -1
> > > > > > > then remote retention is also considered as unlimited but user
> should
> > > > > > > be able to set the local.retention.ms.
> > > > > > > So, we need to introduce local.log.retention.ms and
> > > > > > > local.log.retention.bytes which should  always  be <=
> > > > > > > log.retention.ms/bytes respectively.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <ju...@confluent.io>
> wrote:
> > > > > > > >
> > > > > > > > Hi, Satish,
> > > > > > > >
> > > > > > > > Thanks for the reply. A few quick comments below.
> > > > > > > >
> > > > > > > > 1001. I was thinking that you could just use the tiered
> metadata to
> > > > > do
> > > > > > > the
> > > > > > > > reconciliation. The tiered metadata contains offset ranges
> and
> > > > epoch
> > > > > > > > history. Those should be enough for reconciliation purposes.
> > > > > > > >
> > > > > > > > 1003.3 Having just a serverEndpoint string is probably not
> enough.
> > > > > > > > Connecting to a Kafka cluster may need various security
> > > > credentials.
> > > > > We
> > > > > > > can
> > > > > > > > make RLMM configurable and pass in the properties through the
> > > > > > configure()
> > > > > > > > method. Ditto for RSM.
> > > > > > > >
> > > > > > > > 1008.1 We started with log.retention.hours and
> > > > log.retention.minutes,
> > > > > > and
> > > > > > > > added log.retention.ms later. If we are adding a new
> > > > configuration,
> > > > > ms
> > > > > > > > level config alone is enough and is simpler. We can build
> tools to
> > > > > make
> > > > > > > the
> > > > > > > > configuration at different granularities easier. The
> definition of
> > > > > > > > log.retention.ms is "The number of milliseconds to keep a
> log file
> > > > > > > before
> > > > > > > > deleting it". The deletion is independent of whether tiering
> is
> > > > > enabled
> > > > > > > or
> > > > > > > > not. If this changes to just the local portion of the data,
> we are
> > > > > > > changing
> > > > > > > > the meaning of an existing configuration.
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > > > satish.duggana@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Jun,
> > > > > > > > >
> > > > > > > > > Thank you for the comments! Ying, Harsha and I discussed
> and put
> > > > > our
> > > > > > > > > comments below.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > > > elections.
> > > > > > > This
> > > > > > > > > is very useful, but I am wondering if this is the best
> approach.
> > > > My
> > > > > > > > > understanding of the proposed approach is to allow the new
> > > > > (unclean)
> > > > > > > leader
> > > > > > > > > to take new messages immediately. While this increases
> > > > > availability,
> > > > > > it
> > > > > > > > > creates the problem that there could be multiple
> conflicting
> > > > > segments
> > > > > > > in
> > > > > > > > > the remote store for the same offset range. This seems to
> make it
> > > > > > > harder
> > > > > > > > > for RLMM to determine which archived log segments contain
> the
> > > > > correct
> > > > > > > data.
> > > > > > > > > For example, an archived log segment could at one time be
> the
> > > > > correct
> > > > > > > data,
> > > > > > > > > but be changed to incorrect data after an unclean leader
> > > > election.
> > > > > An
> > > > > > > > > alternative approach is to let the unclean leader use the
> > > > archived
> > > > > > > data as
> > > > > > > > > the source of truth. So, when the new (unclean) leader
> takes
> > > > over,
> > > > > it
> > > > > > > first
> > > > > > > > > reconciles the local data based on the archived data before
> > > > taking
> > > > > > new
> > > > > > > > > messages. This makes the job of RLMM a bit easier since all
> > > > > archived
> > > > > > > data
> > > > > > > > > are considered correct. This increases availability a bit.
> > > > However,
> > > > > > > since
> > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > >
> > > > > > > > > Firstly, We don't want to assume the remote storage is more
> > > > > reliable
> > > > > > > than
> > > > > > > > > Kafka. Kafka unclean leader election usually happens when
> there
> > > > is
> > > > > a
> > > > > > > large
> > > > > > > > > scale outage that impacts multiple racks (or even multiple
> > > > > > availability
> > > > > > > > > zones). In such a case, the remote storage may be
> unavailable or
> > > > > > > unstable.
> > > > > > > > > Pulling a large amount of data from the remote storage to
> > > > reconcile
> > > > > > the
> > > > > > > > > local data may also exacerbate the outage. With the current
> > > > design,
> > > > > > > the new
> > > > > > > > > leader can start working even when the remote storage is
> > > > > temporarily
> > > > > > > > > unavailable.
> > > > > > > > >
> > > > > > > > > Secondly, it is not easier to implement the reconciling
> logic at
> > > > > the
> > > > > > > leader
> > > > > > > > > side. It can take a long time for the new leader to
> download the
> > > > > > remote
> > > > > > > > > data and rebuild local producer id / leader epoch
> information.
> > > > > During
> > > > > > > this
> > > > > > > > > period, the leader cannot accept any requests from the
> clients
> > > > and
> > > > > > > > > followers. We have to introduce a new state for the
> leader, and a
> > > > > new
> > > > > > > error
> > > > > > > > > code to let the clients / followers know what is happening.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > RemoteStorageManager.
> > > > > > > We
> > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all other
> > > > methods,
> > > > > > we
> > > > > > > pass
> > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > >
> > > > > > > > > Nice catch, we can have the RemoteLogSegmentMetadata for
> > > > > > copyLogSegment
> > > > > > > > > too.
> > > > > > > > >
> > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata inclusive
> or
> > > > > > exclusive?
> > > > > > > > >
> > > > > > > > > It is inclusive.
> > > > > > > > >
> > > > > > > > > 1002.3 It seems that we need an api to get the leaderEpoch
> > > > history
> > > > > > for
> > > > > > > a
> > > > > > > > > partition.
> > > > > > > > >
> > > > > > > > > Yes, updated the KIP with the new method.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 1002.4 Could you define the type of
> RemoteLogSegmentContext?
> > > > > > > > >
> > > > > > > > > This is removed in the latest code and it is not needed.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > >
> > > > > > > > > 1003.1 I am not sure why we need both of the following
> methods
> > > > > > > > > in RemoteLogMetadataManager. Could we combine them into
> one that
> > > > > > takes
> > > > > > > in
> > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > >     RemoteLogSegmentId getRemoteLogSegmentId(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > > long offset) throws IOException;
> > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > >
> > > > > > > > > Good point, these can be merged for now. I guess we needed
> them
> > > > in
> > > > > > > earlier
> > > > > > > > > version of the implementation but it is not needed now.
> > > > > > > > >
> > > > > > > > > 1003.2 There seems to be some inconsistencies in the
> methods
> > > > > below. I
> > > > > > > am
> > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the
> other
> > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > >     void deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > >
> > > > > > > > > RLMM stores RemoteLogSegmentMetadata which is identified by
> > > > > > > > > RemoteLogsSegmentId. So, when it is added it takes
> > > > > > > > > RemoteLogSegmentMetadata. `delete` operation needs only
> > > > > > > RemoteLogsSegmentId
> > > > > > > > > as RemoteLogSegmentMetadata can be identified with
> > > > > > RemoteLogsSegmentId.
> > > > > > > > >
> > > > > > > > > 1003.3 In void onServerStarted(final String
> serverEndpoint), what
> > > > > > > > > is serverEndpoint used for?
> > > > > > > > >
> > > > > > > > > This can be used by RLMM implementation to connect to the
> local
> > > > > Kafka
> > > > > > > > > cluster. Incase of default  implementation, it is used in
> > > > > > initializing
> > > > > > > > > kafka clients connecting to the local cluster.
> > > > > > > > >
> > > > > > > > > 1004. It would be useful to document how all the new APIs
> are
> > > > being
> > > > > > > used.
> > > > > > > > > For example, when is
> RemoteLogSegmentMetadata.markedForDeletion
> > > > > being
> > > > > > > set
> > > > > > > > > and used? How are
> > > > > > > > >
> RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > > > > used?
> > > > > > > > >
> > > > > > > > > RLMM APIs are going through the changes and they should be
> ready
> > > > > in a
> > > > > > > few
> > > > > > > > > days. I will update the KIP and the mail  thread once they
> are
> > > > > ready.
> > > > > > > > >
> > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM will
> > > > > eventually
> > > > > > > > > delete these segments by using RemoteStorageManager." Which
> > > > replica
> > > > > > > does
> > > > > > > > > this logic?
> > > > > > > > >
> > > > > > > > > This is a good point. When a topic is deleted, it will not
> have
> > > > any
> > > > > > > > > leader/followers to do the cleanup. We will have a cleaner
> agent
> > > > > on a
> > > > > > > > > single broker in the cluster to do this cleanup, we plan
> to add
> > > > > that
> > > > > > in
> > > > > > > > > controller broker.
> > > > > > > > >
> > > > > > > > > 1006. "If there are any failures in removing remote log
> segments
> > > > > then
> > > > > > > those
> > > > > > > > > are stored in a specific topic (default as
> > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > and user can consume the events(which contain
> > > > > remote-log-segment-id)
> > > > > > > from
> > > > > > > > > that topic and clean them up from remote storage.  " Not
> sure if
> > > > > it's
> > > > > > > worth
> > > > > > > > > the complexity of adding another topic. Could we just
> retry?
> > > > > > > > >
> > > > > > > > > Sure, we can keep this simpler for now by logging an error
> after
> > > > > > > retries.
> > > > > > > > > We can give users a better way to process this in future.
> Oneway
> > > > > can
> > > > > > > be a
> > > > > > > > > dead letter topic which can be configured by the user.
> > > > > > > > >
> > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> existing
> > > > > > > > > fetchPurgatory?
> > > > > > > > >
> > > > > > > > > We have 2 types of delayed operations waiting for 2
> different
> > > > > events.
> > > > > > > > > DelayedFetch waits for new messages from producers.
> > > > > > DelayedRemoteFetch
> > > > > > > > > waits for the remote-storage-read-task to finish. When
> either of
> > > > > the
> > > > > > 2
> > > > > > > > > events happens, we only want to notify one type of the
> delayed
> > > > > > > operations.
> > > > > > > > > It would be inefficient to put 2 types of delayed
> operations in
> > > > one
> > > > > > > > > purgatory, as the tryComplete() methods of the delayed
> operations
> > > > > can
> > > > > > > be
> > > > > > > > > triggered by irrelevant events.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 1008. Configurations:
> > > > > > > > > 1008.1 remote.log.retention.ms,
> remote.log.retention.minutes,
> > > > > > > > > remote.log.retention.hours: It seems that we just need the
> ms
> > > > one.
> > > > > > > Also,
> > > > > > > > > are we changing the meaning of existing config
> log.retention.ms
> > > > to
> > > > > > > mean
> > > > > > > > > the
> > > > > > > > > local retention? For backward compatibility, it's better
> to not
> > > > > > change
> > > > > > > the
> > > > > > > > > meaning of existing configurations.
> > > > > > > > >
> > > > > > > > > We agree that we only need remote.log.retention.ms. But,
> the
> > > > > > existing
> > > > > > > > > Kafka
> > > > > > > > > configuration
> > > > > > > > > has 3 properties (log.retention.ms, log.retention.minutes,
> > > > > > > > > log.retention.hours). We just
> > > > > > > > > want to keep consistent with the existing properties.
> > > > > > > > > Existing log.retention.xxxx config is about log retention
> in
> > > > > broker’s
> > > > > > > > > storage which is local. It should be easy for users to
> configure
> > > > > > > partition
> > > > > > > > > storage with local retention and remote retention based on
> their
> > > > > > usage.
> > > > > > > > >
> > > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic
> level?
> > > > > > > > >
> > > > > > > > > We can introduce topic level config for the same remote.log
> > > > > settings.
> > > > > > > User
> > > > > > > > > can set the desired config while creating the topic.
> > > > > > > > > remote.log.storage.enable property is not allowed to be
> updated
> > > > > after
> > > > > > > the
> > > > > > > > > topic is created. Other remote.log.* properties can be
> modified.
> > > > We
> > > > > > > will
> > > > > > > > > support flipping remote.log.storage.enable in next
> versions.
> > > > > > > > >
> > > > > > > > > 1009. It would be useful to list all limitations in a
> separate
> > > > > > section:
> > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic from
> delete
> > > > > to
> > > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > > >
> > > > > > > > > +1 to have limitations in a separate section. We will
> update the
> > > > > KIP
> > > > > > > with
> > > > > > > > > that.
> > > > > > > > > Topic  created with effective value for remote.log.enabled
> as
> > > > true,
> > > > > > > can not
> > > > > > > > > change its retention policy from delete to compact.
> > > > > > > > >
> > > > > > > > > 1010. Thanks for performance numbers. Are those with
> RocksDB as
> > > > the
> > > > > > > cache?
> > > > > > > > >
> > > > > > > > > No, We have not yet added RocksDB support. This is based on
> > > > > in-memory
> > > > > > > map
> > > > > > > > > representation. We will add that support and update this
> thread
> > > > > after
> > > > > > > > > updating the KIP with the numbers.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Satish.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > > > >
> > > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > > >
> > > > > > > > > > 1000. Regarding Colin's question on querying the metadata
> > > > > directly
> > > > > > > in the
> > > > > > > > > > remote block store. One issue is that not all block
> stores
> > > > offer
> > > > > > the
> > > > > > > > > needed
> > > > > > > > > > api to query the metadata. For example, S3 only offers
> an api
> > > > to
> > > > > > list
> > > > > > > > > > objects under a prefix and this api has the eventual
> > > > consistency
> > > > > > > > > semantic.
> > > > > > > > > >
> > > > > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > > > > elections.
> > > > > > > This
> > > > > > > > > > is very useful, but I am wondering if this is the best
> > > > approach.
> > > > > My
> > > > > > > > > > understanding of the proposed approach is to allow the
> new
> > > > > > (unclean)
> > > > > > > > > leader
> > > > > > > > > > to take new messages immediately. While this increases
> > > > > > availability,
> > > > > > > it
> > > > > > > > > > creates the problem that there could be multiple
> conflicting
> > > > > > > segments in
> > > > > > > > > > the remote store for the same offset range. This seems
> to make
> > > > it
> > > > > > > harder
> > > > > > > > > > for RLMM to determine which archived log segments
> contain the
> > > > > > correct
> > > > > > > > > data.
> > > > > > > > > > For example, an archived log segment could at one time
> be the
> > > > > > correct
> > > > > > > > > data,
> > > > > > > > > > but be changed to incorrect data after an unclean leader
> > > > > election.
> > > > > > An
> > > > > > > > > > alternative approach is to let the unclean leader use the
> > > > > archived
> > > > > > > data
> > > > > > > > > as
> > > > > > > > > > the source of truth. So, when the new (unclean) leader
> takes
> > > > > over,
> > > > > > it
> > > > > > > > > first
> > > > > > > > > > reconciles the local data based on the archived data
> before
> > > > > taking
> > > > > > > new
> > > > > > > > > > messages. This makes the job of RLMM a bit easier since
> all
> > > > > > archived
> > > > > > > data
> > > > > > > > > > are considered correct. This increases availability a
> bit.
> > > > > However,
> > > > > > > since
> > > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > > >
> > > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > > RemoteStorageManager. We
> > > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all
> other
> > > > > methods,
> > > > > > > we
> > > > > > > > > pass
> > > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata
> inclusive or
> > > > > > > exclusive?
> > > > > > > > > > 1002.3 It seems that we need an api to get the
> leaderEpoch
> > > > > history
> > > > > > > for a
> > > > > > > > > > partition.
> > > > > > > > > > 1002.4 Could you define the type of
> RemoteLogSegmentContext?
> > > > > > > > > >
> > > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > > 1003.1 I am not sure why we need both of the following
> methods
> > > > > > > > > > in RemoteLogMetadataManager. Could we combine them into
> one
> > > > that
> > > > > > > takes in
> > > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > > >     RemoteLogSegmentId
> getRemoteLogSegmentId(TopicPartition
> > > > > > > > > topicPartition,
> > > > > > > > > > long offset) throws IOException;
> > > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > 1003.2 There seems to be some inconsistencies in the
> methods
> > > > > below.
> > > > > > > I am
> > > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the
> other
> > > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > > >     void
> deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > > 1003.3 In void onServerStarted(final String
> serverEndpoint),
> > > > what
> > > > > > > > > > is serverEndpoint used for?
> > > > > > > > > >
> > > > > > > > > > 1004. It would be useful to document how all the new
> APIs are
> > > > > being
> > > > > > > used.
> > > > > > > > > > For example, when is
> RemoteLogSegmentMetadata.markedForDeletion
> > > > > > > being set
> > > > > > > > > > and used? How are
> > > > > > > > > >
> RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > > > being
> > > > > > > used?
> > > > > > > > > >
> > > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM
> will
> > > > > > eventually
> > > > > > > > > > delete these segments by using RemoteStorageManager."
> Which
> > > > > replica
> > > > > > > does
> > > > > > > > > > this logic?
> > > > > > > > > >
> > > > > > > > > > 1006. "If there are any failures in removing remote log
> > > > segments
> > > > > > then
> > > > > > > > > those
> > > > > > > > > > are stored in a specific topic (default as
> > > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > > and user can consume the events(which contain
> > > > > > remote-log-segment-id)
> > > > > > > from
> > > > > > > > > > that topic and clean them up from remote storage.  " Not
> sure
> > > > if
> > > > > > it's
> > > > > > > > > worth
> > > > > > > > > > the complexity of adding another topic. Could we just
> retry?
> > > > > > > > > >
> > > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the
> existing
> > > > > > > > > > fetchPurgatory?
> > > > > > > > > >
> > > > > > > > > > 1008. Configurations:
> > > > > > > > > > 1008.1 remote.log.retention.ms,
> remote.log.retention.minutes,
> > > > > > > > > > remote.log.retention.hours: It seems that we just need
> the ms
> > > > > one.
> > > > > > > Also,
> > > > > > > > > > are we changing the meaning of existing config
> > > > log.retention.ms
> > > > > to
> > > > > > > mean
> > > > > > > > > > the
> > > > > > > > > > local retention? For backward compatibility, it's better
> to not
> > > > > > > change
> > > > > > > > > the
> > > > > > > > > > meaning of existing configurations.
> > > > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic
> level?
> > > > > > > > > >
> > > > > > > > > > 1009. It would be useful to list all limitations in a
> separate
> > > > > > > section:
> > > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic
> from
> > > > delete
> > > > > > to
> > > > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > > > >
> > > > > > > > > > 1010. Thanks for performance numbers. Are those with
> RocksDB as
> > > > > the
> > > > > > > > > cache?
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <
> harsha.ch@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Colin,
> > > > > > > > > > >                Thats not what we said in the previous
> email.
> > > > > RLMM
> > > > > > > is
> > > > > > > > > > > pluggable storage and by running numbers even 1PB data
> you do
> > > > > not
> > > > > > > need
> > > > > > > > > > more
> > > > > > > > > > > than 10GB local storage.
> > > > > > > > > > > If in future this becomes a blocker for any users we
> can
> > > > > revisit
> > > > > > > but
> > > > > > > > > this
> > > > > > > > > > > does not warrant another implementation at this point
> to push
> > > > > the
> > > > > > > data
> > > > > > > > > to
> > > > > > > > > > > remote storage.
> > > > > > > > > > > We can ofcourse implement another RLMM that is
> optional for
> > > > > users
> > > > > > > to
> > > > > > > > > > > configure to push to remote. But that doesn't need to
> be
> > > > > > addressed
> > > > > > > in
> > > > > > > > > > this
> > > > > > > > > > > KIP.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Harsha
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe <
> > > > > cmccabe@apache.org
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi Ying,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the response.
> > > > > > > > > > > >
> > > > > > > > > > > > It sounds like you agree that storing the metadata
> in the
> > > > > > remote
> > > > > > > > > > storage
> > > > > > > > > > > > would be a better design overall.  Given that that's
> true,
> > > > is
> > > > > > > there
> > > > > > > > > any
> > > > > > > > > > > > reason to include the worse implementation based on
> > > > RocksDB?
> > > > > > > > > > > >
> > > > > > > > > > > > Choosing a long-term metadata store is not something
> that
> > > > we
> > > > > > > should
> > > > > > > > > do
> > > > > > > > > > > > lightly.  It can take users years to migrate from
> metadata
> > > > > > store
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > other.  I also don't think it's realistic or
> desirable for
> > > > > > users
> > > > > > > to
> > > > > > > > > > write
> > > > > > > > > > > > their own metadata stores.  Even assuming that they
> could
> > > > do
> > > > > a
> > > > > > > good
> > > > > > > > > job
> > > > > > > > > > > at
> > > > > > > > > > > > this, it would create huge fragmentation in the Kafka
> > > > > > ecosystem.
> > > > > > > > > > > >
> > > > > > > > > > > > best,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng wrote:
> > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Satish and I are still discussing some details
> about how
> > > > to
> > > > > > > handle
> > > > > > > > > > > > > transactions / producer ids. Satish is going to
> make some
> > > > > > minor
> > > > > > > > > > changes
> > > > > > > > > > > > to
> > > > > > > > > > > > > RLMM API and other parts. Other than that, we have
> > > > finished
> > > > > > > > > updating
> > > > > > > > > > > the
> > > > > > > > > > > > KIP
> > > > > > > > > > > > >
> > > > > > > > > > > > > I agree with Colin that the current design of using
> > > > rocksDB
> > > > > > is
> > > > > > > not
> > > > > > > > > > > > > optimal. But this design is simple and should work
> for
> > > > > almost
> > > > > > > all
> > > > > > > > > the
> > > > > > > > > > > > > existing Kafka users. RLMM is a plugin. Users can
> replace
> > > > > > > rocksDB
> > > > > > > > > > with
> > > > > > > > > > > > > their own RLMM implementation, if needed. So, I
> think we
> > > > > can
> > > > > > > keep
> > > > > > > > > > > rocksDB
> > > > > > > > > > > > > for now. What do you think?
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Ying
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> > > > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for the update. It's good to see the
> progress on
> > > > > > this.
> > > > > > > > > > Please
> > > > > > > > > > > > let us
> > > > > > > > > > > > > > know when you are done updating the KIP wiki.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng
> > > > > > > > > <yingz@uber.com.invalid
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Satish and I have added more design details in
> the
> > > > KIP,
> > > > > > > > > including
> > > > > > > > > > > > how to
> > > > > > > > > > > > > > > keep consistency between replicas (especially
> when
> > > > > there
> > > > > > is
> > > > > > > > > > > > leadership
> > > > > > > > > > > > > > > changes / log truncations) and new metrics. We
> also
> > > > > made
> > > > > > > some
> > > > > > > > > > other
> > > > > > > > > > > > minor
> > > > > > > > > > > > > > > changes in the doc. We will finish the KIP
> changes in
> > > > > the
> > > > > > > next
> > > > > > > > > > > > couple of
> > > > > > > > > > > > > > > days. We will let you know when we are done.
> Most of
> > > > > the
> > > > > > > > > changes
> > > > > > > > > > > are
> > > > > > > > > > > > > > > already updated to the wiki KIP. You can take
> a look.
> > > > > But
> > > > > > > it's
> > > > > > > > > > not
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As for the implementation, the code is mostly
> done
> > > > and
> > > > > we
> > > > > > > > > already
> > > > > > > > > > > had
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > feature tests / system tests. I have added the
> > > > > > performance
> > > > > > > test
> > > > > > > > > > > > results
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the KIP. However the recent design changes
> (e.g.
> > > > leader
> > > > > > > epoch
> > > > > > > > > > info
> > > > > > > > > > > > > > > management / log truncation / some of the new
> > > > metrics)
> > > > > > > have not
> > > > > > > > > > > been
> > > > > > > > > > > > > > > implemented yet. It will take about 2 weeks
> for us to
> > > > > > > implement
> > > > > > > > > > > > after you
> > > > > > > > > > > > > > > review and agree with those design changes.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun Rao <
> > > > > jun@confluent.io
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Any new updates on the KIP? This feature is
> one of
> > > > > the
> > > > > > > most
> > > > > > > > > > > > important
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > most requested features in Apache Kafka
> right now.
> > > > It
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > > helpful
> > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > we can make sustained progress on this.
> Could you
> > > > > share
> > > > > > > how
> > > > > > > > > far
> > > > > > > > > > > > along
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > the design/implementation right now? Is there
> > > > > anything
> > > > > > > that
> > > > > > > > > > other
> > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > As for "transactional support" and "follower
> > > > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > > > further comments from me as long as the
> producer
> > > > > state
> > > > > > > and
> > > > > > > > > > leader
> > > > > > > > > > > > epoch
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > be restored properly from the object store
> when
> > > > > needed.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM Satish
> Duggana <
> > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > We did not want to add many implementation
> > > > details
> > > > > in
> > > > > > > the
> > > > > > > > > > KIP.
> > > > > > > > > > > > But we
> > > > > > > > > > > > > > > > > decided to add them in the KIP as appendix
> or
> > > > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > > > follower fetch protocol) to describe the
> flow
> > > > with
> > > > > > the
> > > > > > > main
> > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > That will answer most of the queries. I
> will
> > > > update
> > > > > > on
> > > > > > > this
> > > > > > > > > > > mail
> > > > > > > > > > > > > > > > > thread when the respective sections are
> updated.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM Alexandre
> Dupriez
> > > > > > > > > > > > > > > > > <al...@gmail.com> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > A couple of questions specific to the
> section
> > > > > > > "Follower
> > > > > > > > > > > > > > > > > > Requests/Replication", pages 16:17 in the
> > > > design
> > > > > > > document
> > > > > > > > > > > [1].
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900. It is mentioned that followers fetch
> > > > > auxiliary
> > > > > > > > > states
> > > > > > > > > > > > from the
> > > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900.a Does the consistency model of the
> > > > external
> > > > > > > storage
> > > > > > > > > > > > impacts
> > > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > > > of leader epochs and other auxiliary
> data?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900.b What are the benefits of using a
> > > > mechanism
> > > > > to
> > > > > > > store
> > > > > > > > > > and
> > > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > > the leader epochs which is different
> from other
> > > > > > > metadata
> > > > > > > > > > > > associated
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > tiered segments? What are the benefits of
> > > > > > retrieving
> > > > > > > this
> > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > on-demand from the follower rather than
> relying
> > > > > on
> > > > > > > > > > > propagation
> > > > > > > > > > > > via
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > topic __remote_log_metadata? What are the
> > > > > > advantages
> > > > > > > over
> > > > > > > > > > > > using a
> > > > > > > > > > > > > > > > > > dedicated control structure (e.g. a new
> record
> > > > > > type)
> > > > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > > > this topic? Since in the document,
> different
> > > > > > control
> > > > > > > > > paths
> > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > operating in the system, how are the
> metadata
> > > > > > stored
> > > > > > > in
> > > > > > > > > > > > > > > > > > __remote_log_metadata [which also
> include the
> > > > > epoch
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > which offloaded a segment] and the remote
> > > > > auxiliary
> > > > > > > > > states,
> > > > > > > > > > > > kept in
> > > > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900.c A follower can encounter an
> > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > > > this in response to a Fetch or
> > > > > OffsetForLeaderEpoch
> > > > > > > > > > request?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900.d What happens if, after a follower
> > > > > encountered
> > > > > > > an
> > > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE response,
> its
> > > > > > > attempts to
> > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > leader epochs fail (for instance,
> because the
> > > > > > remote
> > > > > > > > > > storage
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > temporarily unavailable)? Does the
> follower
> > > > > > > fallbacks to
> > > > > > > > > a
> > > > > > > > > > > mode
> > > > > > > > > > > > > > where
> > > > > > > > > > > > > > > > > > it ignores tiered segments, and applies
> > > > > truncation
> > > > > > > using
> > > > > > > > > > only
> > > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > available information? What happens when
> access
> > > > > to
> > > > > > > the
> > > > > > > > > > remote
> > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > is restored? How is the replica lineage
> > > > inferred
> > > > > by
> > > > > > > the
> > > > > > > > > > > remote
> > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > epochs reconciled with the follower's
> replica
> > > > > > > lineage,
> > > > > > > > > > which
> > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > evolved? Does the follower remember
> fetching
> > > > > > > auxiliary
> > > > > > > > > > states
> > > > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > > > in the past and attempt reconciliation?
> Is
> > > > there
> > > > > a
> > > > > > > plan
> > > > > > > > > to
> > > > > > > > > > > > offer
> > > > > > > > > > > > > > > > > > different strategies in this scenario,
> > > > > configurable
> > > > > > > via
> > > > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900.e Is the leader epoch cache
> offloaded with
> > > > > > every
> > > > > > > > > > segment?
> > > > > > > > > > > > Or
> > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > a new checkpoint is detected? If that
> > > > information
> > > > > > is
> > > > > > > not
> > > > > > > > > > > always
> > > > > > > > > > > > > > > > > > offloaded to avoid duplicating data, how
> does
> > > > the
> > > > > > > remote
> > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > satisfy the request to retrieve it?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900.f Since the leader epoch cache
> covers the
> > > > > > entire
> > > > > > > > > > replica
> > > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > > > what happens if, after a leader epoch
> cache
> > > > file
> > > > > is
> > > > > > > > > > offloaded
> > > > > > > > > > > > with
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > given segment, the local epoch cache is
> > > > truncated
> > > > > > > [not
> > > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > a range of offset included in tiered
> segments]?
> > > > > How
> > > > > > > are
> > > > > > > > > > > remote
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > local leader epoch caches kept
> consistent?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900.g Consumer can also use leader
> epochs (e.g.
> > > > > to
> > > > > > > enable
> > > > > > > > > > > > fencing
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > protect against stale leaders). What
> > > > differences
> > > > > > > would
> > > > > > > > > > there
> > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > between consumer and follower fetches?
> > > > > Especially,
> > > > > > > would
> > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > > also fetch leader epoch information from
> the
> > > > > remote
> > > > > > > > > > storage?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 900.h Assume a newly elected leader of a
> > > > > > > topic-partition
> > > > > > > > > > > > detects
> > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > recent segments are available in the
> external
> > > > > > > storage,
> > > > > > > > > with
> > > > > > > > > > > > epochs
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > its local epoch. Does it ignore these
> segments
> > > > > and
> > > > > > > their
> > > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > > epoch-to-offset vectors? Or try to
> reconstruct
> > > > > its
> > > > > > > local
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > lineage based on the data remotely
> available?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55, Satish
> Duggana <
> > > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > > Please let us know if you have any
> comments
> > > > on
> > > > > > > > > > > "transactional
> > > > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > > > and "follower requests/replication"
> mentioned
> > > > > in
> > > > > > > the
> > > > > > > > > > wiki.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25 PM Satish
> > > > Duggana <
> > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks Jun for your comments.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >100. It would be useful to provide
> more
> > > > > > details
> > > > > > > on
> > > > > > > > > how
> > > > > > > > > > > > those
> > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to
> really
> > > > > > assess
> > > > > > > > > > whether
> > > > > > > > > > > > the
> > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > apis are sufficient/redundant. A few
> examples
> > > > > below.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > We will update the wiki and let you
> know.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >100.1 deleteRecords seems to only
> advance
> > > > > the
> > > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > Log. How does that trigger the deletion of
> remote
> > > > > log
> > > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > RLMTask for leader partition
> periodically
> > > > > > checks
> > > > > > > > > > whether
> > > > > > > > > > > > there
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > remote log segments earlier to
> > > > logStartOffset
> > > > > > > and the
> > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > > remote log segment metadata and data
> are
> > > > > > deleted
> > > > > > > by
> > > > > > > > > > using
> > > > > > > > > > > > RLMM
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >100.2 stopReplica with deletion is
> used
> > > > in 2
> > > > > > > cases
> > > > > > > > > (a)
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > reassignment; (b) topic deletion. We only
> want to
> > > > > > > delete
> > > > > > > > > the
> > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > metadata in the second case. Also, in the
> second
> > > > > > case,
> > > > > > > who
> > > > > > > > > > > > initiates
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > deletion of the remote segment since the
> leader
> > > > may
> > > > > > not
> > > > > > > > > > exist?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Right, it is deleted only incase of
> topic
> > > > > > > deletion
> > > > > > > > > > only.
> > > > > > > > > > > We
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > cover
> > > > > > > > > > > > > > > > > > > > the details in the KIP.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >100.3 "LogStartOffset of a topic
> can be
> > > > > either
> > > > > > > in
> > > > > > > > > > local
> > > > > > > > > > > > or in
> > > > > > > > > > > > > > > > > remote storage." If LogStartOffset exists
> in both
> > > > > > > places,
> > > > > > > > > > which
> > > > > > > > > > > > one
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > source of truth?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I meant the logStartOffset can point
> to
> > > > > either
> > > > > > of
> > > > > > > > > local
> > > > > > > > > > > > segment
> > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > remote segment but it is initialised
> and
> > > > > > > maintained
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > Log
> > > > > > > > > > > > > > > > class
> > > > > > > > > > > > > > > > > > > > like now.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >100.4 List<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > topicPartition,
> > > > > > > long
> > > > > > > > > > > > minOffset):
> > > > > > > > > > > > > > > How
> > > > > > > > > > > > > > > > > is minOffset supposed to be used?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Returns list of remote segments,
> sorted by
> > > > > > > baseOffset
> > > > > > > > > > in
> > > > > > > > > > > > > > > ascending
> > > > > > > > > > > > > > > > > > > > order that have baseOffset >= the
> given min
> > > > > > > Offset.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >100.5 When copying a segment to
> remote
> > > > > > storage,
> > > > > > > it
> > > > > > > > > > seems
> > > > > > > > > > > > we
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > calling the same
> RLMM.putRemoteLogSegmentData()
> > > > > twice
> > > > > > > > > before
> > > > > > > > > > > and
> > > > > > > > > > > > > > after
> > > > > > > > > > > > > > > > > copyLogSegment(). Could you explain why?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > This is more about
> prepare/commit/rollback
> > > > as
> > > > > > you
> > > > > > > > > > > > suggested.
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > update the wiki with the new APIs.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >100.6 LogSegmentData includes
> > > > > > leaderEpochCache,
> > > > > > > but
> > > > > > > > > > > there
> > > > > > > > > > > > is
> > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > > in RemoteStorageManager to retrieve it.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Nice catch, copy/paste issue. There
> is an
> > > > API
> > > > > > to
> > > > > > > > > > retrieve
> > > > > > > > > > > > it.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >101. If the __remote_log_metadata
> is for
> > > > > > > production
> > > > > > > > > > > usage,
> > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > you provide more details? For example,
> what is
> > > > the
> > > > > > > schema
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > data
> > > > > > > > > > > > > > > > (both
> > > > > > > > > > > > > > > > > key and value)? How is the topic
> > > > maintained,delete
> > > > > or
> > > > > > > > > > compact?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > It is with delete config and it’s
> retention
> > > > > > > period is
> > > > > > > > > > > > suggested
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > more than the remote retention
> period.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >110. Is the cache implementation in
> > > > > > > > > > > > RemoteLogMetadataManager
> > > > > > > > > > > > > > > meant
> > > > > > > > > > > > > > > > > for production usage? If so, could you
> provide
> > > > more
> > > > > > > details
> > > > > > > > > > on
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > and how/where the data is stored?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > The proposal is to have a cache (with
> > > > default
> > > > > > > > > > > > implementation
> > > > > > > > > > > > > > > backed
> > > > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > > rocksdb) but it will be added in
> later
> > > > > > versions.
> > > > > > > We
> > > > > > > > > > will
> > > > > > > > > > > > add
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > future work items.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >111. "Committed offsets can be
> stored in a
> > > > > > local
> > > > > > > > > > file".
> > > > > > > > > > > > Could
> > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > describe the format of the file and where
> it's
> > > > > > stored?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > We will cover this in the KIP.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >112. Truncation of remote segments
> under
> > > > > > unclean
> > > > > > > > > > leader
> > > > > > > > > > > > > > > election:
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > am not sure who figures out the truncated
> remote
> > > > > > > segments
> > > > > > > > > and
> > > > > > > > > > > how
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > information is propagated to all replicas?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > We will add this in detail in the
> KIP.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >113. "If there are any failures in
> > > > removing
> > > > > > > remote
> > > > > > > > > log
> > > > > > > > > > > > > > segments
> > > > > > > > > > > > > > > > > then those are stored in a specific topic
> > > > (default
> > > > > as
> > > > > > > > > > > > > > > > > __remote_segments_to_be_deleted)". Is it
> > > > necessary
> > > > > to
> > > > > > > add
> > > > > > > > > yet
> > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > internal topic? Could we just keep
> retrying?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > This is not really an internal
> topic, it
> > > > will
> > > > > > be
> > > > > > > > > > exposed
> > > > > > > > > > > > as a
> > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > > configurable topic. After a few
> retries, we
> > > > > > want
> > > > > > > user
> > > > > > > > > > to
> > > > > > > > > > > > know
> > > > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > > > > > the failure so that they can take an
> action
> > > > > > > later by
> > > > > > > > > > > > consuming
> > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > > this topic. We want to keep this
> simple
> > > > > instead
> > > > > > > of
> > > > > > > > > > > retrying
> > > > > > > > > > > > > > > > > > > > continuously and maintaining the
> deletion
> > > > > state
> > > > > > > etc.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >114. "We may not need to copy
> > > > > > > producer-id-snapshot
> > > > > > > > > as
> > > > > > > > > > we
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > copying only segments earlier to
> > > > > last-stable-offset."
> > > > > > > Hmm,
> > > > > > > > > > not
> > > > > > > > > > > > sure
> > > > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > > that. The producer snapshot includes
> things like
> > > > > the
> > > > > > > last
> > > > > > > > > > > > timestamp
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > open producer id and can affect when those
> > > > producer
> > > > > > > ids are
> > > > > > > > > > > > expired.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Sure, this will be added as part of
> the
> > > > > > > > > LogSegmentData.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Fri, May 29, 2020 at 6:39 AM Jun
> Rao <
> > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Made another pass on the wiki. A
> few more
> > > > > > > comments
> > > > > > > > > > > below.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 100. It would be useful to provide
> more
> > > > > > > details on
> > > > > > > > > > how
> > > > > > > > > > > > those
> > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to
> really
> > > > > > assess
> > > > > > > > > > whether
> > > > > > > > > > > > the
> > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > apis are sufficient/redundant. A few
> examples
> > > > > below.
> > > > > > > > > > > > > > > > > > > > > 100.1 deleteRecords seems to only
> advance
> > > > > the
> > > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > Log. How does that trigger the deletion of
> remote
> > > > > log
> > > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > > > 100.2 stopReplica with deletion is
> used
> > > > in
> > > > > 2
> > > > > > > cases
> > > > > > > > > > (a)
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > reassignment; (b) topic deletion. We only
> want to
> > > > > > > delete
> > > > > > > > > the
> > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > metadata in the second case. Also, in the
> second
> > > > > > case,
> > > > > > > who
> > > > > > > > > > > > initiates
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > deletion of the remote segment since the
> leader
> > > > may
> > > > > > not
> > > > > > > > > > exist?
> > > > > > > > > > > > > > > > > > > > > 100.3 "LogStartOffset of a topic
> can be
> > > > > > either
> > > > > > > in
> > > > > > > > > > local
> > > > > > > > > > > > or in
> > > > > > > > > > > > > > > > > remote storage." If LogStartOffset exists
> in both
> > > > > > > places,
> > > > > > > > > > which
> > > > > > > > > > > > one
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > source of truth?
> > > > > > > > > > > > > > > > > > > > > 100.4
> List<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > > topicPartition,
> > > > > > > long
> > > > > > > > > > > > minOffset):
> > > > > > > > > > > > > > > How
> > > > > > > > > > > > > > > > > is minOffset supposed to be used?
> > > > > > > > > > > > > > > > > > > > > 100.5 When copying a segment to
> remote
> > > > > > > storage, it
> > > > > > > > > > > seems
> > > > > > > > > > > > we
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > calling the same
> RLMM.putRemoteLogSegmentData()
> > > > > twice
> > > > > > > > > before
> > > > > > > > > > > and
> > > > > > > > > > > > > > after
> > > > > > > > > > > > > > > > > copyLogSegment(). Could you explain why?
> > > > > > > > > > > > > > > > > > > > > 100.6 LogSegmentData includes
> > > > > > > leaderEpochCache, but
> > > > > > > > > > > > there is
> > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > api in RemoteStorageManager to retrieve it.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 101. If the __remote_log_metadata
> is for
> > > > > > > production
> > > > > > > > > > > > usage,
> > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > you provide more details? For example,
> what is
> > > > the
> > > > > > > schema
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > data
> > > > > > > > > > > > > > > > (both
> > > > > > > > > > > > > > > > > key and value)? How is the topic
> > > > maintained,delete
> > > > > or
> > > > > > > > > > compact?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 110. Is the cache implementation in
> > > > > > > > > > > > RemoteLogMetadataManager
> > > > > > > > > > > > > > > > meant
> > > > > > > > > > > > > > > > > for production usage? If so, could you
> provide
> > > > more
> > > > > > > details
> > > > > > > > > > on
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > and how/where the data is stored?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 111. "Committed offsets can be
> stored in
> > > > a
> > > > > > > local
> > > > > > > > > > file".
> > > > > > > > > > > > Could
> > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > describe the format of the file and where
> it's
> > > > > > stored?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 112. Truncation of remote segments
> under
> > > > > > > unclean
> > > > > > > > > > leader
> > > > > > > > > > > > > > > election:
> > > > > > > > > > > > > > > > > I am not sure who figures out the truncated
> > > > remote
> > > > > > > segments
> > > > > > > > > > and
> > > > > > > > > > > > how
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > information is propagated to all replicas?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 113. "If there are any failures in
> > > > removing
> > > > > > > remote
> > > > > > > > > > log
> > > > > > > > > > > > > > segments
> > > > > > > > > > > > > > > > > then those are stored in a specific topic
> > > > (default
> > > > > as
> > > > > > > > > > > > > > > > > __remote_segments_to_be_deleted)". Is it
> > > > necessary
> > > > > to
> > > > > > > add
> > > > > > > > > yet
> > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > internal topic? Could we just keep
> retrying?
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 114. "We may not need to copy
> > > > > > > producer-id-snapshot
> > > > > > > > > as
> > > > > > > > > > > we
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > copying only segments earlier to
> > > > > last-stable-offset."
> > > > > > > Hmm,
> > > > > > > > > > not
> > > > > > > > > > > > sure
> > > > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > > that. The producer snapshot includes
> things like
> > > > > the
> > > > > > > last
> > > > > > > > > > > > timestamp
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > open producer id and can affect when those
> > > > producer
> > > > > > > ids are
> > > > > > > > > > > > expired.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Thu, May 28, 2020 at 5:38 AM
> Satish
> > > > > > Duggana
> > > > > > > <
> > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >> Gentle reminder. Please go
> through the
> > > > > > updated
> > > > > > > > > wiki
> > > > > > > > > > > and
> > > > > > > > > > > > let
> > > > > > > > > > > > > > us
> > > > > > > > > > > > > > > > > know your comments.
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> Thanks,
> > > > > > > > > > > > > > > > > > > > >> Satish.
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> On Tue, May 19, 2020 at 3:50 PM
> Satish
> > > > > > > Duggana <
> > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > > > >>> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >>> Please go through the wiki which
> has
> > > > the
> > > > > > > latest
> > > > > > > > > > > > updates.
> > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > doc is updated frequently to be in sync
> with
> > > > wiki.
> > > > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > > > >>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>> Satish.
> > > > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > > > >>> On Tue, May 19, 2020 at 12:30 AM
> Jun
> > > > Rao
> > > > > <
> > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > > > >>>> Hi, Satish,
> > > > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > > > >>>> Thanks for the update. Just to
> > > > clarify.
> > > > > > > Which
> > > > > > > > > doc
> > > > > > > > > > > has
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > latest updates, the wiki or the google doc?
> > > > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > > > >>>> Jun
> > > > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > > > >>>> On Thu, May 14, 2020 at 10:38 AM
> > > > Satish
> > > > > > > Duggana
> > > > > > > > > <
> > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >>>>> Thanks for your comments.  We
> updated
> > > > > the
> > > > > > > KIP
> > > > > > > > > > with
> > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > details.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> >100. For each of the
> operations
> > > > > related
> > > > > > to
> > > > > > > > > > > tiering,
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > be useful to provide a description on how
> it
> > > > works
> > > > > > > with the
> > > > > > > > > > new
> > > > > > > > > > > > API.
> > > > > > > > > > > > > > > > These
> > > > > > > > > > > > > > > > > include things like consumer fetch, replica
> > > > fetch,
> > > > > > > > > > > > > > offsetForTimestamp,
> > > > > > > > > > > > > > > > > retention (remote and local) by size, time
> and
> > > > > > > > > > logStartOffset,
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > deletion, etc. This will tell us if the
> proposed
> > > > > APIs
> > > > > > > are
> > > > > > > > > > > > sufficient.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> We addressed most of these
> APIs in
> > > > the
> > > > > > > KIP. We
> > > > > > > > > > can
> > > > > > > > > > > > add
> > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > details if needed.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> >101. For the default
> implementation
> > > > > > based
> > > > > > > on
> > > > > > > > > > > > internal
> > > > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > > is it meant as a proof of concept or for
> > > > production
> > > > > > > usage?
> > > > > > > > > I
> > > > > > > > > > > > assume
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > it's the former. However, if it's the
> latter,
> > > > then
> > > > > > the
> > > > > > > KIP
> > > > > > > > > > > needs
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > describe the design in more detail.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> It is production usage as was
> > > > mentioned
> > > > > > in
> > > > > > > an
> > > > > > > > > > > earlier
> > > > > > > > > > > > > > mail.
> > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > plan to update this section in the next
> few days.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> >102. When tiering a segment,
> the
> > > > > segment
> > > > > > > is
> > > > > > > > > > first
> > > > > > > > > > > > > > written
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > the object store and then its metadata is
> written
> > > > > to
> > > > > > > RLMM
> > > > > > > > > > using
> > > > > > > > > > > > the
> > > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > > "void putRemoteLogSegmentData()". One
> potential
> > > > > issue
> > > > > > > with
> > > > > > > > > > this
> > > > > > > > > > > > > > > approach
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > that if the system fails after the first
> > > > operation,
> > > > > > it
> > > > > > > > > > leaves a
> > > > > > > > > > > > > > garbage
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the object store that's never reclaimed.
> One way
> > > > to
> > > > > > > improve
> > > > > > > > > > > this
> > > > > > > > > > > > is
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > two separate APIs, sth like
> > > > > > > > > preparePutRemoteLogSegmentData()
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > > commitPutRemoteLogSegmentData().
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> That is a good point. We
> currently
> > > > > have a
> > > > > > > > > > different
> > > > > > > > > > > > way
> > > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > > > markers in the segment but your suggestion
> is
> > > > much
> > > > > > > better.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> >103. It seems that the
> transactional
> > > > > > > support
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > > > > ability
> > > > > > > > > > > > > > > > > to read from follower are missing.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> KIP is updated with
> transactional
> > > > > > support,
> > > > > > > > > > follower
> > > > > > > > > > > > fetch
> > > > > > > > > > > > > > > > > semantics, and reading from a follower.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> >104. It would be useful to
> provide a
> > > > > > > testing
> > > > > > > > > > plan
> > > > > > > > > > > > for
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> We added a few tests by
> introducing
> > > > > test
> > > > > > > util
> > > > > > > > > for
> > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > storage in the PR. We will provide the
> testing
> > > > plan
> > > > > > in
> > > > > > > the
> > > > > > > > > > next
> > > > > > > > > > > > few
> > > > > > > > > > > > > > > days.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>>>> Satish.
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > > >>>>> On Wed, Feb 26, 2020 at 9:43 PM
> > > > Harsha
> > > > > > > > > > > Chintalapani <
> > > > > > > > > > > > > > > > > kafka@harsha.io> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>> On Tue, Feb 25, 2020 at 12:46
> PM,
> > > > Jun
> > > > > > Rao
> > > > > > > <
> > > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the updated doc.
> The new
> > > > > API
> > > > > > > seems
> > > > > > > > > > to
> > > > > > > > > > > > be an
> > > > > > > > > > > > > > > > > improvement overall. A few more comments
> below.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 100. For each of the
> operations
> > > > > related
> > > > > > > to
> > > > > > > > > > > > tiering, it
> > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > be useful to provide a description on how
> it
> > > > works
> > > > > > > with the
> > > > > > > > > > new
> > > > > > > > > > > > API.
> > > > > > > > > > > > > > > > These
> > > > > > > > > > > > > > > > > include things like consumer fetch, replica
> > > > fetch,
> > > > > > > > > > > > > > offsetForTimestamp,
> > > > > > > > > > > > > > > > > retention
> > > > > > > > > > > > > > > > > > > > >>>>>>> (remote and local) by size,
> time
> > > > and
> > > > > > > > > > > > logStartOffset,
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > deletion, etc. This will tell us if the
> proposed
> > > > > APIs
> > > > > > > are
> > > > > > > > > > > > sufficient.
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>> Thanks for the feedback Jun.
> We will
> > > > > add
> > > > > > > more
> > > > > > > > > > > > details
> > > > > > > > > > > > > > > around
> > > > > > > > > > > > > > > > > this.
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 101. For the default
> implementation
> > > > > > > based on
> > > > > > > > > > > > internal
> > > > > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > > is it meant as a proof of concept or for
> > > > production
> > > > > > > usage?
> > > > > > > > > I
> > > > > > > > > > > > assume
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > it's the former. However, if it's the
> latter,
> > > > then
> > > > > > the
> > > > > > > KIP
> > > > > > > > > > > needs
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > describe the design in more detail.
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>> Yes it meant to be for
> production
> > > > use.
> > > > > > > > > Ideally
> > > > > > > > > > it
> > > > > > > > > > > > would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > good to merge this in as the default
> > > > implementation
> > > > > > for
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > service.
> > > > > > > > > > > > > > > > > We can add more details around design and
> > > > testing.
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 102. When tiering a segment,
> the
> > > > > > segment
> > > > > > > is
> > > > > > > > > > first
> > > > > > > > > > > > > > written
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > the object store and then its metadata is
> written
> > > > > to
> > > > > > > RLMM
> > > > > > > > > > using
> > > > > > > > > > > > the
> > > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > > "void putRemoteLogSegmentData()".
> > > > > > > > > > > > > > > > > > > > >>>>>>> One potential issue with this
> > > > > approach
> > > > > > is
> > > > > > > > > that
> > > > > > > > > > if
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > fails after the first operation, it leaves
> a
> > > > > garbage
> > > > > > > in the
> > > > > > > > > > > > object
> > > > > > > > > > > > > > > store
> > > > > > > > > > > > > > > > > that's never reclaimed. One way to improve
> this
> > > > is
> > > > > to
> > > > > > > have
> > > > > > > > > > two
> > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > APIs, sth like
> preparePutRemoteLogSegmentData()
> > > > and
> > > > > > > > > > > > > > > > > commitPutRemoteLogSegmentData().
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 103. It seems that the
> > > > transactional
> > > > > > > support
> > > > > > > > > > and
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > ability
> > > > > > > > > > > > > > > > > to read from follower are missing.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 104. It would be useful to
> provide
> > > > a
> > > > > > > testing
> > > > > > > > > > plan
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>> We are working on adding more
> > > > details
> > > > > > > around
> > > > > > > > > > > > > > transactional
> > > > > > > > > > > > > > > > > support and coming up with test plan.
> > > > > > > > > > > > > > > > > > > > >>>>>> Add system tests and
> integration
> > > > > tests.
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Mon, Feb 24, 2020 at 8:10
> AM
> > > > > Satish
> > > > > > > > > Duggana
> > > > > > > > > > <
> > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Please look at the earlier
> reply
> > > > and
> > > > > > let
> > > > > > > us
> > > > > > > > > > know
> > > > > > > > > > > > your
> > > > > > > > > > > > > > > > > comments.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Satish.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Wed, Feb 12, 2020 at 4:06
> PM
> > > > > Satish
> > > > > > > > > Duggana
> > > > > > > > > > <
> > > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for your comments on
> the
> > > > > > > separation of
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > log
> > > > > > > > > > > > > > > > > metadata storage and remote log storage.
> > > > > > > > > > > > > > > > > > > > >>>>>>> We had a few discussions
> since
> > > > early
> > > > > > Jan
> > > > > > > on
> > > > > > > > > how
> > > > > > > > > > > to
> > > > > > > > > > > > > > > support
> > > > > > > > > > > > > > > > > eventually consistent stores like S3 by
> > > > uncoupling
> > > > > > > remote
> > > > > > > > > log
> > > > > > > > > > > > segment
> > > > > > > > > > > > > > > > > metadata and remote log storage. It is
> written
> > > > with
> > > > > > > details
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > doc
> > > > > > > > > > > > > > > > > here(1). Below is the brief summary of the
> > > > > discussion
> > > > > > > from
> > > > > > > > > > that
> > > > > > > > > > > > doc.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> The current approach
> consists of
> > > > > > pulling
> > > > > > > the
> > > > > > > > > > > > remote log
> > > > > > > > > > > > > > > > > segment metadata from remote log storage
> APIs. It
> > > > > > > worked
> > > > > > > > > fine
> > > > > > > > > > > for
> > > > > > > > > > > > > > > > storages
> > > > > > > > > > > > > > > > > like HDFS. But one of the problems of
> relying on
> > > > > the
> > > > > > > remote
> > > > > > > > > > > > storage
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > maintain metadata is that tiered-storage
> needs to
> > > > > be
> > > > > > > > > strongly
> > > > > > > > > > > > > > > consistent,
> > > > > > > > > > > > > > > > > with an impact not only on the
> metadata(e.g. LIST
> > > > > in
> > > > > > > S3)
> > > > > > > > > but
> > > > > > > > > > > > also on
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > segment data(e.g. GET after a DELETE in
> S3). The
> > > > > cost
> > > > > > > of
> > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > > metadata in remote storage needs to be
> factored
> > > > in.
> > > > > > > This is
> > > > > > > > > > > true
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > case of S3, LIST APIs incur huge costs as
> you
> > > > > raised
> > > > > > > > > earlier.
> > > > > > > > > > > > > > > > > > > > >>>>>>> So, it is good to separate
> the
> > > > remote
> > > > > > > storage
> > > > > > > > > > > from
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > remote log metadata store. We refactored
> the
> > > > > existing
> > > > > > > > > > > > > > > > RemoteStorageManager
> > > > > > > > > > > > > > > > > and introduced RemoteLogMetadataManager.
> Remote
> > > > log
> > > > > > > > > metadata
> > > > > > > > > > > > store
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > give strong consistency semantics but
> remote log
> > > > > > > storage
> > > > > > > > > can
> > > > > > > > > > be
> > > > > > > > > > > > > > > > eventually
> > > > > > > > > > > > > > > > > consistent.
> > > > > > > > > > > > > > > > > > > > >>>>>>> We can have a default
> > > > implementation
> > > > > > for
> > > > > > > > > > > > > > > > > RemoteLogMetadataManager which uses an
> internal
> > > > > > > topic(as
> > > > > > > > > > > > mentioned in
> > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > of our earlier emails) as storage. But
> users can
> > > > > > always
> > > > > > > > > > plugin
> > > > > > > > > > > > their
> > > > > > > > > > > > > > > own
> > > > > > > > > > > > > > > > > RemoteLogMetadataManager implementation
> based on
> > > > > > their
> > > > > > > > > > > > environment.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Please go through the
> updated KIP
> > > > and
> > > > > > > let us
> > > > > > > > > > know
> > > > > > > > > > > > your
> > > > > > > > > > > > > > > > > comments. We have started refactoring for
> the
> > > > > changes
> > > > > > > > > > mentioned
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > and there may be a few more updates to the
> APIs.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> [1]
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> https://docs.google.com/document/d/1qfkBCWL1e7ZWkHU7brxKDBebq4ie9yK20XJnKbgAlew/edit?ts=5e208ec7#
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Dec 27, 2019 at 5:43
> PM
> > > > Ivan
> > > > > > > > > Yurchenko
> > > > > > > > > > <
> > > > > > > > > > > > > > > > > ivan0yurchenko@gmail.com>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi all,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Jun:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (a) Cost: S3 list object
> requests
> > > > > cost
> > > > > > > $0.005
> > > > > > > > > > per
> > > > > > > > > > > > 1000
> > > > > > > > > > > > > > > > > requests. If
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> have 100,000 partitions and
> want to
> > > > > > pull
> > > > > > > the
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> partition
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> at
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the rate of 1/sec. It can
> cost
> > > > > > $0.5/sec,
> > > > > > > > > which
> > > > > > > > > > is
> > > > > > > > > > > > > > roughly
> > > > > > > > > > > > > > > > > $40K per
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> day.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> I want to note here, that no
> > > > > reasonably
> > > > > > > > > durable
> > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > be cheap at 100k RPS. For example,
> DynamoDB might
> > > > > > give
> > > > > > > the
> > > > > > > > > > same
> > > > > > > > > > > > > > > ballpark
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> figures.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> If we want to keep the
> pull-based
> > > > > > > approach,
> > > > > > > > > we
> > > > > > > > > > > can
> > > > > > > > > > > > try
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > reduce this
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> number
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> in several ways: doing
> listings
> > > > less
> > > > > > > > > frequently
> > > > > > > > > > > (as
> > > > > > > > > > > > > > > Satish
> > > > > > > > > > > > > > > > > mentioned, with the current defaults it's
> ~3.33k
> > > > > RPS
> > > > > > > for
> > > > > > > > > your
> > > > > > > > > > > > > > example),
> > > > > > > > > > > > > > > > > batching listing operations in some way
> > > > (depending
> > > > > on
> > > > > > > the
> > > > > > > > > > > > storage; it
> > > > > > > > > > > > > > > > might
> > > > > > > > > > > > > > > > > require the change of RSM's interface).
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> There are different ways for
> doing
> > > > > push
> > > > > > > based
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > propagation.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Some
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> object stores may support
> that
> > > > > already.
> > > > > > > For
> > > > > > > > > > > > example, S3
> > > > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> events
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> notification
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> This sounds interesting.
> However, I
> > > > > > see a
> > > > > > > > > > couple
> > > > > > > > > > > of
> > > > > > > > > > > > > > > issues
> > > > > > > > > > > > > > > > > using it:
> > > > > > > > > > > > > > > > > > > > >>>>>>> 1. As I understand the
> > > > documentation,
> > > > > > > > > > > notification
> > > > > > > > > > > > > > > delivery
> > > > > > > > > > > > > > > > > is not guaranteed
> > > > > > > > > > > > > > > > > > > > >>>>>>> and it's recommended to
> > > > periodically
> > > > > do
> > > > > > > LIST
> > > > > > > > > to
> > > > > > > > > > > > fill
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > gaps. Which brings us back to the same LIST
> > > > > > consistency
> > > > > > > > > > > > guarantees
> > > > > > > > > > > > > > > issue.
> > > > > > > > > > > > > > > > > > > > >>>>>>> 2. The same goes for the
> broker
> > > > > start:
> > > > > > > to get
> > > > > > > > > > the
> > > > > > > > > > > > > > current
> > > > > > > > > > > > > > > > > state, we
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> need
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to LIST.
> > > > > > > > > > > > > > > > > > > > >>>>>>> 3. The dynamic set of
> multiple
> > > > > > consumers
> > > > > > > > > > (RSMs):
> > > > > > > > > > > > AFAIK
> > > > > > > > > > > > > > > SQS
> > > > > > > > > > > > > > > > > and SNS
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> aren't
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> designed for such a case.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Alexandre:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> A.1 As commented on PR 7561,
> S3
> > > > > > > consistency
> > > > > > > > > > model
> > > > > > > > > > > > > > [1][2]
> > > > > > > > > > > > > > > > > implies RSM
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> cannot
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> relies solely on S3 APIs to
> > > > guarantee
> > > > > > the
> > > > > > > > > > > expected
> > > > > > > > > > > > > > strong
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> consistency. The
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> proposed implementation [3]
> would
> > > > > need
> > > > > > > to be
> > > > > > > > > > > > updated to
> > > > > > > > > > > > > > > > take
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> into
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> account. Let’s talk more
> about
> > > > this.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thank you for the feedback. I
> > > > clearly
> > > > > > > see the
> > > > > > > > > > > need
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > changing the S3 implementation
> > > > > > > > > > > > > > > > > > > > >>>>>>> to provide stronger
> consistency
> > > > > > > guarantees.
> > > > > > > > > As
> > > > > > > > > > it
> > > > > > > > > > > > see
> > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > this thread, there are
> > > > > > > > > > > > > > > > > > > > >>>>>>> several possible approaches
> to
> > > > this.
> > > > > > > Let's
> > > > > > > > > > > discuss
> > > > > > > > > > > > > > > > > RemoteLogManager's contract and
> > > > > > > > > > > > > > > > > > > > >>>>>>> behavior (like pull vs push
> model)
> > > > > > > further
> > > > > > > > > > before
> > > > > > > > > > > > > > picking
> > > > > > > > > > > > > > > > > one (or
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> several -
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> ?) of them.
> > > > > > > > > > > > > > > > > > > > >>>>>>> I'm going to do some
> evaluation of
> > > > > > > DynamoDB
> > > > > > > > > for
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > pull-based
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> approach,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> if it's possible to apply it
> > > > paying a
> > > > > > > > > > reasonable
> > > > > > > > > > > > bill.
> > > > > > > > > > > > > > > > Also,
> > > > > > > > > > > > > > > > > of the push-based approach
> > > > > > > > > > > > > > > > > > > > >>>>>>> with a Kafka topic as the
> medium.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> A.2.3 Atomicity – what does
> an
> > > > > > > implementation
> > > > > > > > > > of
> > > > > > > > > > > > RSM
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> respect to atomicity of the
> APIs
> > > > > > > > > > copyLogSegment,
> > > > > > > > > > > > > > > > > cleanupLogUntil and deleteTopicPartition?
> If a
> > > > > > partial
> > > > > > > > > > failure
> > > > > > > > > > > > > > happens
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > any of those
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (e.g.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the S3 implementation, if
> one of
> > > > the
> > > > > > > multiple
> > > > > > > > > > > > uploads
> > > > > > > > > > > > > > > fails
> > > > > > > > > > > > > > > > > [4]),
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> The S3 implementation is
> going to
> > > > > > > change, but
> > > > > > > > > > > it's
> > > > > > > > > > > > > > worth
> > > > > > > > > > > > > > > > > clarifying
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> anyway.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> The segment log file is being
> > > > > uploaded
> > > > > > > after
> > > > > > > > > S3
> > > > > > > > > > > has
> > > > > > > > > > > > > > acked
> > > > > > > > > > > > > > > > > uploading of all other files associated
> with the
> > > > > > > segment
> > > > > > > > > and
> > > > > > > > > > > only
> > > > > > > > > > > > > > after
> > > > > > > > > > > > > > > > > this the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> whole
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> segment file set becomes
> visible
> > > > > > > remotely for
> > > > > > > > > > > > > > operations
> > > > > > > > > > > > > > > > > like listRemoteSegments [1].
> > > > > > > > > > > > > > > > > > > > >>>>>>> In case of upload failure,
> the
> > > > files
> > > > > > > that has
> > > > > > > > > > > been
> > > > > > > > > > > > > > > > > successfully
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> uploaded
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> stays
> > > > > > > > > > > > > > > > > > > > >>>>>>> as invisible garbage that is
> > > > > collected
> > > > > > by
> > > > > > > > > > > > > > cleanupLogUntil
> > > > > > > > > > > > > > > > (or
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> overwritten
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> successfully later).
> > > > > > > > > > > > > > > > > > > > >>>>>>> And the opposite happens
> during the
> > > > > > > deletion:
> > > > > > > > > > log
> > > > > > > > > > > > files
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > deleted
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> first.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> This approach should
> generally work
> > > > > > when
> > > > > > > we
> > > > > > > > > > solve
> > > > > > > > > > > > > > > > > consistency issues by adding a strongly
> > > > consistent
> > > > > > > > > storage: a
> > > > > > > > > > > > > > segment's
> > > > > > > > > > > > > > > > > uploaded files
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> remain
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> invisible garbage until some
> > > > metadata
> > > > > > > about
> > > > > > > > > > them
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > written.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> A.3 Caching – storing
> locally the
> > > > > > > segments
> > > > > > > > > > > > retrieved
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > the remote storage is excluded as it does
> not
> > > > align
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > > original
> > > > > > > > > > > > > > > > intent
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> and even
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> defeat some of its purposes
> (save
> > > > > disk
> > > > > > > space
> > > > > > > > > > > etc.).
> > > > > > > > > > > > > > That
> > > > > > > > > > > > > > > > > said, could
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> there
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> be other types of use cases
> where
> > > > the
> > > > > > > pattern
> > > > > > > > > > of
> > > > > > > > > > > > access
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> remotely
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> stored segments would
> benefit from
> > > > > > local
> > > > > > > > > > caching
> > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > potentially read-ahead)? Consider the use
> case
> > > > of a
> > > > > > > large
> > > > > > > > > > pool
> > > > > > > > > > > of
> > > > > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> start
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> a backfill at the same time
> for one
> > > > > day
> > > > > > > worth
> > > > > > > > > > of
> > > > > > > > > > > > data
> > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > one year
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> ago
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> stored remotely. Caching the
> > > > segments
> > > > > > > locally
> > > > > > > > > > > would
> > > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> uncouple the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> load on the remote storage
> from the
> > > > > > load
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > cluster. Maybe
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> RLM could expose a
> configuration
> > > > > > > parameter to
> > > > > > > > > > > > switch
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > feature
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> on/off?
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> I tend to agree here, caching
> > > > remote
> > > > > > > segments
> > > > > > > > > > > > locally
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > making this configurable sounds pretty
> practical
> > > > to
> > > > > > > me. We
> > > > > > > > > > > should
> > > > > > > > > > > > > > > > implement
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> this,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> maybe not in the first
> iteration.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Br,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Ivan
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> [1]
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> https://github.com/harshach/kafka/pull/18/files#diff-4d73d01c16caed6f2548fc3063550ef0R152
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Thu, 19 Dec 2019 at 19:49,
> > > > > Alexandre
> > > > > > > > > > Dupriez <
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> alexandre.dupriez@gmail.com>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thank you for the feedback.
> I am
> > > > > trying
> > > > > > > to
> > > > > > > > > > > > understand
> > > > > > > > > > > > > > > how a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> push-based
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> approach would work.
> > > > > > > > > > > > > > > > > > > > >>>>>>> In order for the metadata to
> be
> > > > > > > propagated
> > > > > > > > > > (under
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > assumption you stated), would you plan to
> add a
> > > > new
> > > > > > > API in
> > > > > > > > > > > Kafka
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > > > the metadata store to send them directly
> to the
> > > > > > > brokers?
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Alexandre
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Le mer. 18 déc. 2019 à
> 20:14, Jun
> > > > > Rao <
> > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > écrit :
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Ivan,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> There are different ways for
> doing
> > > > > push
> > > > > > > based
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> propagation. Some
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> object stores may support
> that
> > > > > already.
> > > > > > > For
> > > > > > > > > > > > example, S3
> > > > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> events
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> notification (
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > >
> > > >
> https://docs.aws.amazon.com/AmazonS3/latest/dev/NotificationHowTo.html
> > > > > > > > > > > > > > > ).
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Otherwise one could use a
> separate
> > > > > > > metadata
> > > > > > > > > > store
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> push-based
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> change propagation. Other
> people
> > > > have
> > > > > > > > > mentioned
> > > > > > > > > > > > using a
> > > > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> topic. The
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> best approach may depend on
> the
> > > > > object
> > > > > > > store
> > > > > > > > > > and
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > operational environment (e.g. whether an
> external
> > > > > > > metadata
> > > > > > > > > > > store
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> available).
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> The above discussion is
> based on
> > > > the
> > > > > > > > > assumption
> > > > > > > > > > > > that we
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> cache the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> object metadata locally in
> every
> > > > > > broker.
> > > > > > > I
> > > > > > > > > > > > mentioned
> > > > > > > > > > > > > > > > earlier
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> an
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> alternative is to just
> > > > store/retrieve
> > > > > > > those
> > > > > > > > > > > > metadata in
> > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > external metadata store. That may simplify
> the
> > > > > > > > > implementation
> > > > > > > > > > > in
> > > > > > > > > > > > some
> > > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Dec 5, 2019 at 7:01
> AM
> > > > Satish
> > > > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for your reply.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Currently,
> `listRemoteSegments` is
> > > > > > > called at
> > > > > > > > > > the
> > > > > > > > > > > > > > > configured
> > > > > > > > > > > > > > > > > interval(not every second, defaults to
> 30secs).
> > > > > > Storing
> > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> log
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> metadata in a strongly
> consistent
> > > > > store
> > > > > > > for
> > > > > > > > > S3
> > > > > > > > > > > RSM
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > raised
> > > > > > > > > > > > > > > > > in PR-comment[1].
> > > > > > > > > > > > > > > > > > > > >>>>>>> RLM invokes RSM at regular
> > > > intervals
> > > > > > and
> > > > > > > RSM
> > > > > > > > > > can
> > > > > > > > > > > > give
> > > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> segment
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> metadata if it is available.
> RSM is
> > > > > > > > > responsible
> > > > > > > > > > > for
> > > > > > > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> fetching those entries. It
> should
> > > > be
> > > > > > > based on
> > > > > > > > > > > > whatever
> > > > > > > > > > > > > > > > > mechanism
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> consistent and efficient
> with the
> > > > > > > respective
> > > > > > > > > > > remote
> > > > > > > > > > > > > > > > storage.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Can you give more details
> about
> > > > push
> > > > > > > based
> > > > > > > > > > > > mechanism
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > RSM?
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 1.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > https://github.com/apache/kafka/pull/7561#discussion_r344576223
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Satish.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Dec 5, 2019 at 4:23
> AM Jun
> > > > > Rao
> > > > > > <
> > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Harsha,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 40/41. I am curious which
> block
> > > > > > storages
> > > > > > > you
> > > > > > > > > > have
> > > > > > > > > > > > > > tested.
> > > > > > > > > > > > > > > > S3
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> seems
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to be
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> one of the popular block
> stores.
> > > > The
> > > > > > > concerns
> > > > > > > > > > > that
> > > > > > > > > > > > I
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> pull
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> based
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> approach are the following.
> > > > > > > > > > > > > > > > > > > > >>>>>>> (a) Cost: S3 list object
> requests
> > > > > cost
> > > > > > > $0.005
> > > > > > > > > > per
> > > > > > > > > > > > 1000
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> requests. If
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> have 100,000 partitions and
> want to
> > > > > > pull
> > > > > > > the
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> partition
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> at
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the rate of 1/sec. It can
> cost
> > > > > > $0.5/sec,
> > > > > > > > > which
> > > > > > > > > > is
> > > > > > > > > > > > > > roughly
> > > > > > > > > > > > > > > > > $40K
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> per
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> day.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (b) Semantics: S3 list
> objects are
> > > > > > > eventually
> > > > > > > > > > > > > > consistent.
> > > > > > > > > > > > > > > > So,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> when
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> do a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> list object request, there
> is no
> > > > > > > guarantee
> > > > > > > > > that
> > > > > > > > > > > > you can
> > > > > > > > > > > > > > > see
> > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> uploaded
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> objects. This could impact
> the
> > > > > > > correctness of
> > > > > > > > > > > > > > subsequent
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> logics.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (c) Efficiency: Blindly
> pulling
> > > > > > metadata
> > > > > > > when
> > > > > > > > > > > > there is
> > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> change adds
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> unnecessary overhead in the
> broker
> > > > as
> > > > > > > well as
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > block
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> store.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> So, have you guys tested S3?
> If so,
> > > > > > > could you
> > > > > > > > > > > share
> > > > > > > > > > > > > > your
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> experience
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> terms of cost, semantics and
> > > > > > efficiency?
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Dec 3, 2019 at 10:11
> PM
> > > > > Harsha
> > > > > > > > > > > > Chintalapani <
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> kafka@harsha.io
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 26, 2019 at 3:46
> PM,
> > > > Jun
> > > > > > Rao
> > > > > > > <
> > > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Ying,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 40/41. There are two
> different ways
> > > > > > that
> > > > > > > we
> > > > > > > > > can
> > > > > > > > > > > > > > approach
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> this.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> One is
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> what
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> you said. We can have an
> > > > opinionated
> > > > > > way
> > > > > > > of
> > > > > > > > > > > > storing and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> populating
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> metadata that we think is
> good
> > > > enough
> > > > > > for
> > > > > > > > > > > > everyone. I
> > > > > > > > > > > > > > am
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> sure if
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> is the case based on what's
> > > > currently
> > > > > > > > > proposed
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> For
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> example, I
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> am not sure that (1) everyone
> > > > always
> > > > > > > needs
> > > > > > > > > > local
> > > > > > > > > > > > > > > metadata;
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (2)
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> current
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> local storage format is
> general
> > > > > enough
> > > > > > > and
> > > > > > > > > (3)
> > > > > > > > > > > > everyone
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wants to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> use
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> pull based approach to
> propagate
> > > > the
> > > > > > > > > metadata.
> > > > > > > > > > > > Another
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> approach
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> is to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> make
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> this pluggable and let the
> > > > > implementor
> > > > > > > > > > implements
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > best
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> approach
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> for a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> particular block storage. I
> haven't
> > > > > > seen
> > > > > > > any
> > > > > > > > > > > > comments
> > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Slack/AirBnb
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the mailing list on this
> topic. It
> > > > > > would
> > > > > > > be
> > > > > > > > > > great
> > > > > > > > > > > > if
> > > > > > > > > > > > > > they
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> provide
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> feedback directly here.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> The current interfaces are
> designed
> > > > > > with
> > > > > > > most
> > > > > > > > > > > > popular
> > > > > > > > > > > > > > > block
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storages
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> available today and we did 2
> > > > > > > implementations
> > > > > > > > > > with
> > > > > > > > > > > > these
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> interfaces and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> they both are yielding good
> results
> > > > > as
> > > > > > we
> > > > > > > > > going
> > > > > > > > > > > > through
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> testing of
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> it.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> If there is ever a need for
> pull
> > > > > based
> > > > > > > > > approach
> > > > > > > > > > > we
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> definitely
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> evolve
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the interface.
> > > > > > > > > > > > > > > > > > > > >>>>>>> In the past we did mark
> interfaces
> > > > to
> > > > > > be
> > > > > > > > > > evolving
> > > > > > > > > > > > to
> > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> room for
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> unknowns
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> in the future.
> > > > > > > > > > > > > > > > > > > > >>>>>>> If you have any suggestions
> around
> > > > > the
> > > > > > > > > current
> > > > > > > > > > > > > > interfaces
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> please
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> propose we
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> are happy to see if we can
> work
> > > > them
> > > > > > > into it.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 43. To offer tier storage as
> a
> > > > > general
> > > > > > > > > feature,
> > > > > > > > > > > > ideally
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> existing
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> capabilities should still be
> > > > > supported.
> > > > > > > It's
> > > > > > > > > > fine
> > > > > > > > > > > > if
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> uber
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> implementation doesn't
> support all
> > > > > > > > > capabilities
> > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> internal
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> usage.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> However, the framework
> should be
> > > > > > general
> > > > > > > > > > enough.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> We agree on that as a
> principle.
> > > > But
> > > > > > all
> > > > > > > of
> > > > > > > > > > these
> > > > > > > > > > > > major
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> features
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> mostly
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> coming right now and to have
> a new
> > > > > big
> > > > > > > > > feature
> > > > > > > > > > > > such as
> > > > > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> support all the new features
> will
> > > > be
> > > > > a
> > > > > > > big
> > > > > > > > > ask.
> > > > > > > > > > > We
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> document on
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> how
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> do
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> we approach solving these in
> future
> > > > > > > > > iterations.
> > > > > > > > > > > > > > > > > > > > >>>>>>> Our goal is to make this
> tiered
> > > > > storage
> > > > > > > > > feature
> > > > > > > > > > > > work
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> everyone.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 43.3 This is more than just
> serving
> > > > > the
> > > > > > > > > tier-ed
> > > > > > > > > > > > data
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> block
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> With KIP-392, the consumer
> now can
> > > > > > > resolve
> > > > > > > > > the
> > > > > > > > > > > > > > conflicts
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> with the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> replica
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> based on leader epoch. So,
> we need
> > > > to
> > > > > > > make
> > > > > > > > > sure
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> leader epoch
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> can be
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> recovered properly from tier
> > > > storage.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> We are working on testing our
> > > > > approach
> > > > > > > and we
> > > > > > > > > > > will
> > > > > > > > > > > > > > update
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the KIP
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> design details.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 43.4 For JBOD, if tier
> storage
> > > > stores
> > > > > > the
> > > > > > > > > tier
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> locally, we
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> need to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> support moving such metadata
> across
> > > > > > disk
> > > > > > > > > > > > directories
> > > > > > > > > > > > > > > since
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> JBOD
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> supports
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> moving data across disks.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> KIP is updated with JBOD
> details.
> > > > > > Having
> > > > > > > said
> > > > > > > > > > > that
> > > > > > > > > > > > JBOD
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> tooling
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> needs
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> evolve to support production
> loads.
> > > > > > Most
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > users
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> interested in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> using tiered storage without
> JBOD
> > > > > > support
> > > > > > > > > > support
> > > > > > > > > > > > on
> > > > > > > > > > > > > > day
> > > > > > > > > > > > > > > 1.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>>>>>> Harsha
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> As for meeting, we could
> have a KIP
> > > > > > > e-meeting
> > > > > > > > > > on
> > > > > > > > > > > > this
> > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> needed,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> but it
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> will be open to everyone and
> will
> > > > be
> > > > > > > recorded
> > > > > > > > > > and
> > > > > > > > > > > > > > shared.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Often,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> details are still resolved
> through
> > > > > the
> > > > > > > > > mailing
> > > > > > > > > > > > list.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 19, 2019 at 6:48
> PM
> > > > Ying
> > > > > > > Zheng
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> <yi...@uber.com.invalid>
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Please ignore my previous
> email
> > > > > > > > > > > > > > > > > > > > >>>>>>> I didn't know Apache
> requires all
> > > > the
> > > > > > > > > > discussions
> > > > > > > > > > > > to be
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> "open"
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 19, 2019, 5:40
> PM Ying
> > > > > > Zheng
> > > > > > > <
> > > > > > > > > > > > > > yingz@uber.com
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thank you very much for your
> > > > > feedback!
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Can we schedule a meeting in
> your
> > > > > Palo
> > > > > > > Alto
> > > > > > > > > > > office
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> December? I
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> think a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> face to face discussion is
> much
> > > > more
> > > > > > > > > efficient
> > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> emails. Both
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Harsha
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> I can visit you. Satish may
> be able
> > > > > to
> > > > > > > join
> > > > > > > > > us
> > > > > > > > > > > > > > remotely.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Nov 15, 2019 at
> 11:04 AM
> > > > Jun
> > > > > > Rao
> > > > > > > <
> > > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Harsha,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> The following is a more
> detailed
> > > > high
> > > > > > > level
> > > > > > > > > > > > feedback
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the KIP.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Overall,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the KIP seems useful. The
> challenge
> > > > > is
> > > > > > > how to
> > > > > > > > > > > > design it
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> such that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> it’s
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> general enough to support
> different
> > > > > > ways
> > > > > > > of
> > > > > > > > > > > > > > implementing
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> feature
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> support existing features.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 40. Local segment metadata
> storage:
> > > > > The
> > > > > > > KIP
> > > > > > > > > > makes
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> assumption
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> metadata for the archived log
> > > > > segments
> > > > > > > are
> > > > > > > > > > cached
> > > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> every
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> broker
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> and provides a specific
> > > > > implementation
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> framework. We probably should
> > > > discuss
> > > > > > > this
> > > > > > > > > > more.
> > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> example,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> some
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage providers may not
> want to
> > > > > cache
> > > > > > > the
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> locally and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> just
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> rely
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> upon a remote key/value
> store if
> > > > > such a
> > > > > > > store
> > > > > > > > > > is
> > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> present. If
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> local store is used, there
> could be
> > > > > > > different
> > > > > > > > > > > ways
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> implementing it
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (e.g., based on customized
> local
> > > > > files,
> > > > > > > an
> > > > > > > > > > > embedded
> > > > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> store
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> like
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> RocksDB, etc). An
> alternative of
> > > > > > > designing
> > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> provide an
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> interface for retrieving the
> tier
> > > > > > segment
> > > > > > > > > > > metadata
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> leave the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> details
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> of
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> how to get the metadata
> outside of
> > > > > the
> > > > > > > > > > framework.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 41. RemoteStorageManager
> interface
> > > > > and
> > > > > > > the
> > > > > > > > > > usage
> > > > > > > > > > > > of the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> interface in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> framework: I am not sure if
> the
> > > > > > > interface is
> > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> enough. For
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> example,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> it seems that
> RemoteLogIndexEntry
> > > > is
> > > > > > > tied to
> > > > > > > > > a
> > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> way of
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storing
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> metadata in remote storage.
> The
> > > > > > framework
> > > > > > > > > uses
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> listRemoteSegments()
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> api
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> a pull based approach.
> However, in
> > > > > some
> > > > > > > other
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> implementations, a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> push
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> based
> > > > > > > > > > > > > > > > > > > > >>>>>>> approach may be more
> preferred. I
> > > > > don’t
> > > > > > > have
> > > > > > > > > a
> > > > > > > > > > > > concrete
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> proposal
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> yet.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> But,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> it would be useful to give
> this
> > > > area
> > > > > > some
> > > > > > > > > more
> > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> and see
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> if we
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> make the interface more
> general.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 42. In the diagram, the
> > > > > > RemoteLogManager
> > > > > > > is
> > > > > > > > > > side
> > > > > > > > > > > by
> > > > > > > > > > > > > > side
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> LogManager.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> This KIP only discussed how
> the
> > > > fetch
> > > > > > > request
> > > > > > > > > > is
> > > > > > > > > > > > > > handled
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> between
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> two
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> layer. However, we should
> also
> > > > > consider
> > > > > > > how
> > > > > > > > > > other
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> touch
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> log can be handled. e.g.,
> list
> > > > > offsets
> > > > > > by
> > > > > > > > > > > > timestamp,
> > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> records,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> etc.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Also, in this model, it's
> not clear
> > > > > > which
> > > > > > > > > > > > component is
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> responsible
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> for
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> managing the log start
> offset. It
> > > > > seems
> > > > > > > that
> > > > > > > > > > the
> > > > > > > > > > > > log
> > > > > > > > > > > > > > > start
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> offset
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> could
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> be
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> changed by both
> RemoteLogManager
> > > > and
> > > > > > > > > > LogManager.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 43. There are quite a few
> existing
> > > > > > > features
> > > > > > > > > not
> > > > > > > > > > > > covered
> > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> KIP.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> It
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> would be useful to discuss
> each of
> > > > > > those.
> > > > > > > > > > > > > > > > > > > > >>>>>>> 43.1 I won’t say that
> compacted
> > > > > topics
> > > > > > > are
> > > > > > > > > > rarely
> > > > > > > > > > > > used
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> always
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> small.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> For example, KStreams uses
> > > > compacted
> > > > > > > topics
> > > > > > > > > for
> > > > > > > > > > > > storing
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> states
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> sometimes the size of the
> topic
> > > > could
> > > > > > be
> > > > > > > > > large.
> > > > > > > > > > > > While
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> might
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> be ok
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> support compacted topics
> initially,
> > > > > it
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > > useful
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> have a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> high
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> level
> > > > > > > > > > > > > > > > > > > > >>>>>>> idea on how this might be
> supported
> > > > > > down
> > > > > > > the
> > > > > > > > > > road
> > > > > > > > > > > > so
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> we
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> don’t
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> have
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> make incompatible API
> changes in
> > > > the
> > > > > > > future.
> > > > > > > > > > > > > > > > > > > > >>>>>>> 43.2 We need to discuss how
> EOS is
> > > > > > > supported.
> > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> particular, how
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> producer state integrated
> with the
> > > > > > remote
> > > > > > > > > > > storage.
> > > > > > > > > > > > 43.3
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Now that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> KIP-392
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (allow consumers to fetch
> from
> > > > > closest
> > > > > > > > > replica)
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> implemented,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> we
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> need
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> discuss how reading from a
> follower
> > > > > > > replica
> > > > > > > > > is
> > > > > > > > > > > > > > supported
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> 43.4 We need to discuss how
> JBOD is
> > > > > > > supported
> > > > > > > > > > > with
> > > > > > > > > > > > tier
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Nov 8, 2019 at 12:06
> AM Tom
> > > > > > > Bentley <
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> tbentley@redhat.com
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for those insights
> Ying.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Nov 7, 2019 at 9:26
> PM Ying
> > > > > > Zheng
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> <yingz@uber.com.invalid
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Thanks, I missed that point.
> > > > However,
> > > > > > > there's
> > > > > > > > > > > > still a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> point at
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> which
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> consumer fetches start
> getting
> > > > served
> > > > > > > from
> > > > > > > > > > remote
> > > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (even
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> if
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> point isn't as soon as the
> local
> > > > log
> > > > > > > > > retention
> > > > > > > > > > > > > > > time/size).
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> This
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> represents
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> a kind of performance cliff
> edge
> > > > and
> > > > > > > what I'm
> > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> interested
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> how
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> easy it is for a consumer
> which
> > > > falls
> > > > > > off
> > > > > > > > > that
> > > > > > > > > > > > cliff to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> catch up
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> and so
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> its
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> fetches again come from local
> > > > > storage.
> > > > > > > > > > Obviously
> > > > > > > > > > > > this
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> depend
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> on
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> all
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> sorts of factors (like
> production
> > > > > rate,
> > > > > > > > > > > consumption
> > > > > > > > > > > > > > > rate),
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> so
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> it's
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> guaranteed (just like it's
> not
> > > > > > > guaranteed for
> > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> today), but
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> would
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> represent a new failure mode.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> As I have explained in the
> last
> > > > mail,
> > > > > > > it's a
> > > > > > > > > > very
> > > > > > > > > > > > rare
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> case that
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> a
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> consumer
> > > > > > > > > > > > > > > > > > > > >>>>>>> need to read remote data.
> With our
> > > > > > > experience
> > > > > > > > > > at
> > > > > > > > > > > > Uber,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> this only
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> happens
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> when the consumer service
> had an
> > > > > outage
> > > > > > > for
> > > > > > > > > > > several
> > > > > > > > > > > > > > > hours.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> There is not a "performance
> cliff"
> > > > as
> > > > > > you
> > > > > > > > > > assume.
> > > > > > > > > > > > The
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> even faster than local disks
> in
> > > > terms
> > > > > > of
> > > > > > > > > > > bandwidth.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Reading from
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage is going to have
> higher
> > > > > latency
> > > > > > > than
> > > > > > > > > > > local
> > > > > > > > > > > > > > disk.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> But
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> since
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> consumer
> > > > > > > > > > > > > > > > > > > > >>>>>>> is catching up several hours
> data,
> > > > > it's
> > > > > > > not
> > > > > > > > > > > > sensitive
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> sub-second
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> level
> > > > > > > > > > > > > > > > > > > > >>>>>>> latency, and each remote read
> > > > request
> > > > > > > will
> > > > > > > > > > read a
> > > > > > > > > > > > large
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> amount of
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> data to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> make the overall performance
> better
> > > > > > than
> > > > > > > > > > reading
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> disks.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Another aspect I'd like to
> > > > understand
> > > > > > > better
> > > > > > > > > is
> > > > > > > > > > > the
> > > > > > > > > > > > > > > effect
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> of
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> serving
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> fetch
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> request from remote storage
> has on
> > > > > the
> > > > > > > > > broker's
> > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> utilization. If
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> we're just trimming the
> amount of
> > > > > data
> > > > > > > held
> > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> (without
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> increasing
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> overall local+remote
> retention),
> > > > then
> > > > > > > we're
> > > > > > > > > > > > effectively
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> trading
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> disk
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> bandwidth for network
> bandwidth
> > > > when
> > > > > > > serving
> > > > > > > > > > > fetch
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> requests from
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage (which I understand
> to be a
> > > > > > good
> > > > > > > > > thing,
> > > > > > > > > > > > since
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> brokers are
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> often/usually disk bound).
> But if
> > > > > we're
> > > > > > > > > > > increasing
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> overall
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> local+remote
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> retention then it's more
> likely
> > > > that
> > > > > > > network
> > > > > > > > > > > itself
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> becomes the
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> bottleneck.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> I appreciate this is all
> rather
> > > > hand
> > > > > > > wavy,
> > > > > > > > > I'm
> > > > > > > > > > > just
> > > > > > > > > > > > > > > trying
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> understand
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> how this would affect broker
> > > > > > > performance, so
> > > > > > > > > > I'd
> > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> grateful for
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> any
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> insights you can offer.
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> Network bandwidth is a
> function of
> > > > > > > produce
> > > > > > > > > > speed,
> > > > > > > > > > > > it
> > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> nothing
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> do
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> remote retention. As long as
> the
> > > > data
> > > > > > is
> > > > > > > > > > shipped
> > > > > > > > > > > to
> > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> storage,
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> keep the data there for 1
> day or 1
> > > > > year
> > > > > > > or
> > > > > > > > > 100
> > > > > > > > > > > > years,
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> doesn't
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> consume
> > > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>> any
> > > > > > > > > > > > > > > > > > > > >>>>>>> network resources.
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
>
>

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

Posted by Satish Duggana <sa...@gmail.com>.
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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition

Satish.


On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana <sa...@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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition
>
>
> Satish.
>
>
> On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng <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 <ju...@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 <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
> > > > unclean-leader-election.
> > > > 2) Since the user set local retention time (or local retention bytes), I
> > > > think we are expected to
> > > > keep that much local data when possible (avoid truncating all the local
> > > > data). But, as you said,
> > > > unclean leader elections are very rare, this may not be a big problem.
> > > >
> > > > The current design uses the leader broker as source-of-truth. This is
> > > > consistent with the
> > > > existing Kafka behavior.
> > > >
> > > > By using remote storage as the source-of-truth, the follower logic can
> > > be a
> > > > little simpler,
> > > > but the leader logic is going to be more complex. Overall, I don't see
> > > > there many benefits
> > > > of using remote storage as the source-of-truth.
> > > >
> > > >
> > > >
> > > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Hi, Satish,
> > > > >
> > > > > Thanks for the reply.
> > > > >
> > > > > 1001. In your example, I was thinking that you could just download the
> > > > > latest leader epoch from the object store. After that you know the
> > > leader
> > > > > should end with offset 1100. The leader will delete all its local data
> > > > > before offset 1000 and start accepting new messages at offset 1100.
> > > > > Consumer requests for messages before offset 1100 will be served from
> > > the
> > > > > object store. The benefit with this approach is that it's simpler to
> > > > reason
> > > > > about who is the source of truth. The downside is slightly  increased
> > > > > unavailability window during unclean leader election. Since unclean
> > > > leader
> > > > > elections are rare, I am not sure if this is a big concern.
> > > > >
> > > > > 1008. Yes, I think introducing sth like local.retention.ms seems more
> > > > > consistent.
> > > > >
> > > > > Jun
> > > > >
> > > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > > satish.duggana@gmail.com
> > > > >
> > > > > wrote:
> > > > >
> > > > > > HI Jun,
> > > > > > Thanks for your comments. We put our inline replies below.
> > > > > >
> > > > > > 1001. I was thinking that you could just use the tiered metadata to
> > > do
> > > > > the
> > > > > > reconciliation. The tiered metadata contains offset ranges and epoch
> > > > > > history. Those should be enough for reconciliation purposes.
> > > > > >
> > > > > > If we use remote storage as the source-of-truth during
> > > > > > unclean-leader-election, it's possible that after reconciliation the
> > > > > > remote storage will have more recent data than the new leader's local
> > > > > > storage. For example, the new leader's latest message is offset 1000,
> > > > > > while the remote storage has message 1100. In such a case, the new
> > > > > > leader will have to download the messages from 1001 to 1100, before
> > > > > > accepting new messages from producers. Otherwise, there would be a
> > > gap
> > > > > > in the local data between 1000 and 1101.
> > > > > >
> > > > > > Moreover, with the current design, leader epoch history is stored in
> > > > > > remote storage, rather than the metadata topic. We did consider
> > > saving
> > > > > > epoch history in remote segment metadata. But the concern is that
> > > > > > there is currently no limit for the epoch history size.
> > > Theoretically,
> > > > > > if a user has a very long remote retention time and there are very
> > > > > > frequent leadership changes, the leader epoch history can become too
> > > > > > long to fit into a regular Kafka message.
> > > > > >
> > > > > >
> > > > > > 1003.3 Having just a serverEndpoint string is probably not enough.
> > > > > > Connecting to a Kafka cluster may need various security credentials.
> > > We
> > > > > can
> > > > > > make RLMM configurable and pass in the properties through the
> > > > configure()
> > > > > > method. Ditto for RSM.
> > > > > >
> > > > > > RLMM and  RSM are already configurable and they take properties which
> > > > > > start with "remote.log.metadata." and "remote.log.storage."
> > > > > > respectively and a few others. We have listener-name as the config
> > > for
> > > > > > RLMM and other properties(like security) can be sent as you
> > > suggested.
> > > > > > We will update the KIP with the details.
> > > > > >
> > > > > >
> > > > > > 1008.1 We started with log.retention.hours and log.retention.minutes,
> > > > and
> > > > > > added log.retention.ms later. If we are adding a new configuration,
> > > ms
> > > > > > level config alone is enough and is simpler. We can build tools to
> > > make
> > > > > the
> > > > > > configuration at different granularities easier. The definition of
> > > > > > log.retention.ms is "The number of milliseconds to keep a log file
> > > > > before
> > > > > > deleting it". The deletion is independent of whether tiering is
> > > enabled
> > > > > or
> > > > > > not. If this changes to just the local portion of the data, we are
> > > > > changing
> > > > > > the meaning of an existing configuration.
> > > > > >
> > > > > > We are fine with either way. We can go with log.retention.xxxx as the
> > > > > > effective log retention instead of local log retention. With this
> > > > > > convention, we need to introduce  local.log.retention instead of
> > > > > > remote.log.retention.ms that we proposed. If log.retention.ms as -1
> > > > > > then remote retention is also considered as unlimited but user should
> > > > > > be able to set the local.retention.ms.
> > > > > > So, we need to introduce local.log.retention.ms and
> > > > > > local.log.retention.bytes which should  always  be <=
> > > > > > log.retention.ms/bytes respectively.
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > > >
> > > > > > > Hi, Satish,
> > > > > > >
> > > > > > > Thanks for the reply. A few quick comments below.
> > > > > > >
> > > > > > > 1001. I was thinking that you could just use the tiered metadata to
> > > > do
> > > > > > the
> > > > > > > reconciliation. The tiered metadata contains offset ranges and
> > > epoch
> > > > > > > history. Those should be enough for reconciliation purposes.
> > > > > > >
> > > > > > > 1003.3 Having just a serverEndpoint string is probably not enough.
> > > > > > > Connecting to a Kafka cluster may need various security
> > > credentials.
> > > > We
> > > > > > can
> > > > > > > make RLMM configurable and pass in the properties through the
> > > > > configure()
> > > > > > > method. Ditto for RSM.
> > > > > > >
> > > > > > > 1008.1 We started with log.retention.hours and
> > > log.retention.minutes,
> > > > > and
> > > > > > > added log.retention.ms later. If we are adding a new
> > > configuration,
> > > > ms
> > > > > > > level config alone is enough and is simpler. We can build tools to
> > > > make
> > > > > > the
> > > > > > > configuration at different granularities easier. The definition of
> > > > > > > log.retention.ms is "The number of milliseconds to keep a log file
> > > > > > before
> > > > > > > deleting it". The deletion is independent of whether tiering is
> > > > enabled
> > > > > > or
> > > > > > > not. If this changes to just the local portion of the data, we are
> > > > > > changing
> > > > > > > the meaning of an existing configuration.
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > > satish.duggana@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Jun,
> > > > > > > >
> > > > > > > > Thank you for the comments! Ying, Harsha and I discussed and put
> > > > our
> > > > > > > > comments below.
> > > > > > > >
> > > > > > > >
> > > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > > elections.
> > > > > > This
> > > > > > > > is very useful, but I am wondering if this is the best approach.
> > > My
> > > > > > > > understanding of the proposed approach is to allow the new
> > > > (unclean)
> > > > > > leader
> > > > > > > > to take new messages immediately. While this increases
> > > > availability,
> > > > > it
> > > > > > > > creates the problem that there could be multiple conflicting
> > > > segments
> > > > > > in
> > > > > > > > the remote store for the same offset range. This seems to make it
> > > > > > harder
> > > > > > > > for RLMM to determine which archived log segments contain the
> > > > correct
> > > > > > data.
> > > > > > > > For example, an archived log segment could at one time be the
> > > > correct
> > > > > > data,
> > > > > > > > but be changed to incorrect data after an unclean leader
> > > election.
> > > > An
> > > > > > > > alternative approach is to let the unclean leader use the
> > > archived
> > > > > > data as
> > > > > > > > the source of truth. So, when the new (unclean) leader takes
> > > over,
> > > > it
> > > > > > first
> > > > > > > > reconciles the local data based on the archived data before
> > > taking
> > > > > new
> > > > > > > > messages. This makes the job of RLMM a bit easier since all
> > > > archived
> > > > > > data
> > > > > > > > are considered correct. This increases availability a bit.
> > > However,
> > > > > > since
> > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > >
> > > > > > > > Firstly, We don't want to assume the remote storage is more
> > > > reliable
> > > > > > than
> > > > > > > > Kafka. Kafka unclean leader election usually happens when there
> > > is
> > > > a
> > > > > > large
> > > > > > > > scale outage that impacts multiple racks (or even multiple
> > > > > availability
> > > > > > > > zones). In such a case, the remote storage may be unavailable or
> > > > > > unstable.
> > > > > > > > Pulling a large amount of data from the remote storage to
> > > reconcile
> > > > > the
> > > > > > > > local data may also exacerbate the outage. With the current
> > > design,
> > > > > > the new
> > > > > > > > leader can start working even when the remote storage is
> > > > temporarily
> > > > > > > > unavailable.
> > > > > > > >
> > > > > > > > Secondly, it is not easier to implement the reconciling logic at
> > > > the
> > > > > > leader
> > > > > > > > side. It can take a long time for the new leader to download the
> > > > > remote
> > > > > > > > data and rebuild local producer id / leader epoch information.
> > > > During
> > > > > > this
> > > > > > > > period, the leader cannot accept any requests from the clients
> > > and
> > > > > > > > followers. We have to introduce a new state for the leader, and a
> > > > new
> > > > > > error
> > > > > > > > code to let the clients / followers know what is happening.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > RemoteStorageManager.
> > > > > > We
> > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all other
> > > methods,
> > > > > we
> > > > > > pass
> > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > >
> > > > > > > > Nice catch, we can have the RemoteLogSegmentMetadata for
> > > > > copyLogSegment
> > > > > > > > too.
> > > > > > > >
> > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata inclusive or
> > > > > exclusive?
> > > > > > > >
> > > > > > > > It is inclusive.
> > > > > > > >
> > > > > > > > 1002.3 It seems that we need an api to get the leaderEpoch
> > > history
> > > > > for
> > > > > > a
> > > > > > > > partition.
> > > > > > > >
> > > > > > > > Yes, updated the KIP with the new method.
> > > > > > > >
> > > > > > > >
> > > > > > > > 1002.4 Could you define the type of RemoteLogSegmentContext?
> > > > > > > >
> > > > > > > > This is removed in the latest code and it is not needed.
> > > > > > > >
> > > > > > > >
> > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > >
> > > > > > > > 1003.1 I am not sure why we need both of the following methods
> > > > > > > > in RemoteLogMetadataManager. Could we combine them into one that
> > > > > takes
> > > > > > in
> > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > >     RemoteLogSegmentId getRemoteLogSegmentId(TopicPartition
> > > > > > topicPartition,
> > > > > > > > long offset) throws IOException;
> > > > > > > >     RemoteLogSegmentMetadata
> > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > >
> > > > > > > > Good point, these can be merged for now. I guess we needed them
> > > in
> > > > > > earlier
> > > > > > > > version of the implementation but it is not needed now.
> > > > > > > >
> > > > > > > > 1003.2 There seems to be some inconsistencies in the methods
> > > > below. I
> > > > > > am
> > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the other
> > > > > > > > takes RemoteLogSegmentId.
> > > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > >     void deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > >
> > > > > > > > RLMM stores RemoteLogSegmentMetadata which is identified by
> > > > > > > > RemoteLogsSegmentId. So, when it is added it takes
> > > > > > > > RemoteLogSegmentMetadata. `delete` operation needs only
> > > > > > RemoteLogsSegmentId
> > > > > > > > as RemoteLogSegmentMetadata can be identified with
> > > > > RemoteLogsSegmentId.
> > > > > > > >
> > > > > > > > 1003.3 In void onServerStarted(final String serverEndpoint), what
> > > > > > > > is serverEndpoint used for?
> > > > > > > >
> > > > > > > > This can be used by RLMM implementation to connect to the local
> > > > Kafka
> > > > > > > > cluster. Incase of default  implementation, it is used in
> > > > > initializing
> > > > > > > > kafka clients connecting to the local cluster.
> > > > > > > >
> > > > > > > > 1004. It would be useful to document how all the new APIs are
> > > being
> > > > > > used.
> > > > > > > > For example, when is RemoteLogSegmentMetadata.markedForDeletion
> > > > being
> > > > > > set
> > > > > > > > and used? How are
> > > > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > > > used?
> > > > > > > >
> > > > > > > > RLMM APIs are going through the changes and they should be ready
> > > > in a
> > > > > > few
> > > > > > > > days. I will update the KIP and the mail  thread once they are
> > > > ready.
> > > > > > > >
> > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM will
> > > > eventually
> > > > > > > > delete these segments by using RemoteStorageManager." Which
> > > replica
> > > > > > does
> > > > > > > > this logic?
> > > > > > > >
> > > > > > > > This is a good point. When a topic is deleted, it will not have
> > > any
> > > > > > > > leader/followers to do the cleanup. We will have a cleaner agent
> > > > on a
> > > > > > > > single broker in the cluster to do this cleanup, we plan to add
> > > > that
> > > > > in
> > > > > > > > controller broker.
> > > > > > > >
> > > > > > > > 1006. "If there are any failures in removing remote log segments
> > > > then
> > > > > > those
> > > > > > > > are stored in a specific topic (default as
> > > > > > __remote_segments_to_be_deleted)
> > > > > > > > and user can consume the events(which contain
> > > > remote-log-segment-id)
> > > > > > from
> > > > > > > > that topic and clean them up from remote storage.  " Not sure if
> > > > it's
> > > > > > worth
> > > > > > > > the complexity of adding another topic. Could we just retry?
> > > > > > > >
> > > > > > > > Sure, we can keep this simpler for now by logging an error after
> > > > > > retries.
> > > > > > > > We can give users a better way to process this in future. Oneway
> > > > can
> > > > > > be a
> > > > > > > > dead letter topic which can be configured by the user.
> > > > > > > >
> > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the existing
> > > > > > > > fetchPurgatory?
> > > > > > > >
> > > > > > > > We have 2 types of delayed operations waiting for 2 different
> > > > events.
> > > > > > > > DelayedFetch waits for new messages from producers.
> > > > > DelayedRemoteFetch
> > > > > > > > waits for the remote-storage-read-task to finish. When either of
> > > > the
> > > > > 2
> > > > > > > > events happens, we only want to notify one type of the delayed
> > > > > > operations.
> > > > > > > > It would be inefficient to put 2 types of delayed operations in
> > > one
> > > > > > > > purgatory, as the tryComplete() methods of the delayed operations
> > > > can
> > > > > > be
> > > > > > > > triggered by irrelevant events.
> > > > > > > >
> > > > > > > >
> > > > > > > > 1008. Configurations:
> > > > > > > > 1008.1 remote.log.retention.ms, remote.log.retention.minutes,
> > > > > > > > remote.log.retention.hours: It seems that we just need the ms
> > > one.
> > > > > > Also,
> > > > > > > > are we changing the meaning of existing config log.retention.ms
> > > to
> > > > > > mean
> > > > > > > > the
> > > > > > > > local retention? For backward compatibility, it's better to not
> > > > > change
> > > > > > the
> > > > > > > > meaning of existing configurations.
> > > > > > > >
> > > > > > > > We agree that we only need remote.log.retention.ms. But, the
> > > > > existing
> > > > > > > > Kafka
> > > > > > > > configuration
> > > > > > > > has 3 properties (log.retention.ms, log.retention.minutes,
> > > > > > > > log.retention.hours). We just
> > > > > > > > want to keep consistent with the existing properties.
> > > > > > > > Existing log.retention.xxxx config is about log retention in
> > > > broker’s
> > > > > > > > storage which is local. It should be easy for users to configure
> > > > > > partition
> > > > > > > > storage with local retention and remote retention based on their
> > > > > usage.
> > > > > > > >
> > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic level?
> > > > > > > >
> > > > > > > > We can introduce topic level config for the same remote.log
> > > > settings.
> > > > > > User
> > > > > > > > can set the desired config while creating the topic.
> > > > > > > > remote.log.storage.enable property is not allowed to be updated
> > > > after
> > > > > > the
> > > > > > > > topic is created. Other remote.log.* properties can be modified.
> > > We
> > > > > > will
> > > > > > > > support flipping remote.log.storage.enable in next versions.
> > > > > > > >
> > > > > > > > 1009. It would be useful to list all limitations in a separate
> > > > > section:
> > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic from delete
> > > > to
> > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > >
> > > > > > > > +1 to have limitations in a separate section. We will update the
> > > > KIP
> > > > > > with
> > > > > > > > that.
> > > > > > > > Topic  created with effective value for remote.log.enabled as
> > > true,
> > > > > > can not
> > > > > > > > change its retention policy from delete to compact.
> > > > > > > >
> > > > > > > > 1010. Thanks for performance numbers. Are those with RocksDB as
> > > the
> > > > > > cache?
> > > > > > > >
> > > > > > > > No, We have not yet added RocksDB support. This is based on
> > > > in-memory
> > > > > > map
> > > > > > > > representation. We will add that support and update this thread
> > > > after
> > > > > > > > updating the KIP with the numbers.
> > > > > > > >
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Satish.
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > > >
> > > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > > >
> > > > > > > > > 1000. Regarding Colin's question on querying the metadata
> > > > directly
> > > > > > in the
> > > > > > > > > remote block store. One issue is that not all block stores
> > > offer
> > > > > the
> > > > > > > > needed
> > > > > > > > > api to query the metadata. For example, S3 only offers an api
> > > to
> > > > > list
> > > > > > > > > objects under a prefix and this api has the eventual
> > > consistency
> > > > > > > > semantic.
> > > > > > > > >
> > > > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > > > elections.
> > > > > > This
> > > > > > > > > is very useful, but I am wondering if this is the best
> > > approach.
> > > > My
> > > > > > > > > understanding of the proposed approach is to allow the new
> > > > > (unclean)
> > > > > > > > leader
> > > > > > > > > to take new messages immediately. While this increases
> > > > > availability,
> > > > > > it
> > > > > > > > > creates the problem that there could be multiple conflicting
> > > > > > segments in
> > > > > > > > > the remote store for the same offset range. This seems to make
> > > it
> > > > > > harder
> > > > > > > > > for RLMM to determine which archived log segments contain the
> > > > > correct
> > > > > > > > data.
> > > > > > > > > For example, an archived log segment could at one time be the
> > > > > correct
> > > > > > > > data,
> > > > > > > > > but be changed to incorrect data after an unclean leader
> > > > election.
> > > > > An
> > > > > > > > > alternative approach is to let the unclean leader use the
> > > > archived
> > > > > > data
> > > > > > > > as
> > > > > > > > > the source of truth. So, when the new (unclean) leader takes
> > > > over,
> > > > > it
> > > > > > > > first
> > > > > > > > > reconciles the local data based on the archived data before
> > > > taking
> > > > > > new
> > > > > > > > > messages. This makes the job of RLMM a bit easier since all
> > > > > archived
> > > > > > data
> > > > > > > > > are considered correct. This increases availability a bit.
> > > > However,
> > > > > > since
> > > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > > >
> > > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > > RemoteStorageManager. We
> > > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all other
> > > > methods,
> > > > > > we
> > > > > > > > pass
> > > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata inclusive or
> > > > > > exclusive?
> > > > > > > > > 1002.3 It seems that we need an api to get the leaderEpoch
> > > > history
> > > > > > for a
> > > > > > > > > partition.
> > > > > > > > > 1002.4 Could you define the type of RemoteLogSegmentContext?
> > > > > > > > >
> > > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > > 1003.1 I am not sure why we need both of the following methods
> > > > > > > > > in RemoteLogMetadataManager. Could we combine them into one
> > > that
> > > > > > takes in
> > > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > > >     RemoteLogSegmentId getRemoteLogSegmentId(TopicPartition
> > > > > > > > topicPartition,
> > > > > > > > > long offset) throws IOException;
> > > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > 1003.2 There seems to be some inconsistencies in the methods
> > > > below.
> > > > > > I am
> > > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the other
> > > > > > > > > takes RemoteLogSegmentId.
> > > > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > > >     void deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > > 1003.3 In void onServerStarted(final String serverEndpoint),
> > > what
> > > > > > > > > is serverEndpoint used for?
> > > > > > > > >
> > > > > > > > > 1004. It would be useful to document how all the new APIs are
> > > > being
> > > > > > used.
> > > > > > > > > For example, when is RemoteLogSegmentMetadata.markedForDeletion
> > > > > > being set
> > > > > > > > > and used? How are
> > > > > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > > being
> > > > > > used?
> > > > > > > > >
> > > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM will
> > > > > eventually
> > > > > > > > > delete these segments by using RemoteStorageManager." Which
> > > > replica
> > > > > > does
> > > > > > > > > this logic?
> > > > > > > > >
> > > > > > > > > 1006. "If there are any failures in removing remote log
> > > segments
> > > > > then
> > > > > > > > those
> > > > > > > > > are stored in a specific topic (default as
> > > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > > and user can consume the events(which contain
> > > > > remote-log-segment-id)
> > > > > > from
> > > > > > > > > that topic and clean them up from remote storage.  " Not sure
> > > if
> > > > > it's
> > > > > > > > worth
> > > > > > > > > the complexity of adding another topic. Could we just retry?
> > > > > > > > >
> > > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the existing
> > > > > > > > > fetchPurgatory?
> > > > > > > > >
> > > > > > > > > 1008. Configurations:
> > > > > > > > > 1008.1 remote.log.retention.ms, remote.log.retention.minutes,
> > > > > > > > > remote.log.retention.hours: It seems that we just need the ms
> > > > one.
> > > > > > Also,
> > > > > > > > > are we changing the meaning of existing config
> > > log.retention.ms
> > > > to
> > > > > > mean
> > > > > > > > > the
> > > > > > > > > local retention? For backward compatibility, it's better to not
> > > > > > change
> > > > > > > > the
> > > > > > > > > meaning of existing configurations.
> > > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic level?
> > > > > > > > >
> > > > > > > > > 1009. It would be useful to list all limitations in a separate
> > > > > > section:
> > > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic from
> > > delete
> > > > > to
> > > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > > >
> > > > > > > > > 1010. Thanks for performance numbers. Are those with RocksDB as
> > > > the
> > > > > > > > cache?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <harsha.ch@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Colin,
> > > > > > > > > >                Thats not what we said in the previous email.
> > > > RLMM
> > > > > > is
> > > > > > > > > > pluggable storage and by running numbers even 1PB data you do
> > > > not
> > > > > > need
> > > > > > > > > more
> > > > > > > > > > than 10GB local storage.
> > > > > > > > > > If in future this becomes a blocker for any users we can
> > > > revisit
> > > > > > but
> > > > > > > > this
> > > > > > > > > > does not warrant another implementation at this point to push
> > > > the
> > > > > > data
> > > > > > > > to
> > > > > > > > > > remote storage.
> > > > > > > > > > We can ofcourse implement another RLMM that is optional for
> > > > users
> > > > > > to
> > > > > > > > > > configure to push to remote. But that doesn't need to be
> > > > > addressed
> > > > > > in
> > > > > > > > > this
> > > > > > > > > > KIP.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Harsha
> > > > > > > > > >
> > > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe <
> > > > cmccabe@apache.org
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Ying,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the response.
> > > > > > > > > > >
> > > > > > > > > > > It sounds like you agree that storing the metadata in the
> > > > > remote
> > > > > > > > > storage
> > > > > > > > > > > would be a better design overall.  Given that that's true,
> > > is
> > > > > > there
> > > > > > > > any
> > > > > > > > > > > reason to include the worse implementation based on
> > > RocksDB?
> > > > > > > > > > >
> > > > > > > > > > > Choosing a long-term metadata store is not something that
> > > we
> > > > > > should
> > > > > > > > do
> > > > > > > > > > > lightly.  It can take users years to migrate from metadata
> > > > > store
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > other.  I also don't think it's realistic or desirable for
> > > > > users
> > > > > > to
> > > > > > > > > write
> > > > > > > > > > > their own metadata stores.  Even assuming that they could
> > > do
> > > > a
> > > > > > good
> > > > > > > > job
> > > > > > > > > > at
> > > > > > > > > > > this, it would create huge fragmentation in the Kafka
> > > > > ecosystem.
> > > > > > > > > > >
> > > > > > > > > > > best,
> > > > > > > > > > > Colin
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng wrote:
> > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > Hi Colin,
> > > > > > > > > > > >
> > > > > > > > > > > > Satish and I are still discussing some details about how
> > > to
> > > > > > handle
> > > > > > > > > > > > transactions / producer ids. Satish is going to make some
> > > > > minor
> > > > > > > > > changes
> > > > > > > > > > > to
> > > > > > > > > > > > RLMM API and other parts. Other than that, we have
> > > finished
> > > > > > > > updating
> > > > > > > > > > the
> > > > > > > > > > > KIP
> > > > > > > > > > > >
> > > > > > > > > > > > I agree with Colin that the current design of using
> > > rocksDB
> > > > > is
> > > > > > not
> > > > > > > > > > > > optimal. But this design is simple and should work for
> > > > almost
> > > > > > all
> > > > > > > > the
> > > > > > > > > > > > existing Kafka users. RLMM is a plugin. Users can replace
> > > > > > rocksDB
> > > > > > > > > with
> > > > > > > > > > > > their own RLMM implementation, if needed. So, I think we
> > > > can
> > > > > > keep
> > > > > > > > > > rocksDB
> > > > > > > > > > > > for now. What do you think?
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Ying
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> > > jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the update. It's good to see the progress on
> > > > > this.
> > > > > > > > > Please
> > > > > > > > > > > let us
> > > > > > > > > > > > > know when you are done updating the KIP wiki.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng
> > > > > > > > <yingz@uber.com.invalid
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Satish and I have added more design details in the
> > > KIP,
> > > > > > > > including
> > > > > > > > > > > how to
> > > > > > > > > > > > > > keep consistency between replicas (especially when
> > > > there
> > > > > is
> > > > > > > > > > > leadership
> > > > > > > > > > > > > > changes / log truncations) and new metrics. We also
> > > > made
> > > > > > some
> > > > > > > > > other
> > > > > > > > > > > minor
> > > > > > > > > > > > > > changes in the doc. We will finish the KIP changes in
> > > > the
> > > > > > next
> > > > > > > > > > > couple of
> > > > > > > > > > > > > > days. We will let you know when we are done. Most of
> > > > the
> > > > > > > > changes
> > > > > > > > > > are
> > > > > > > > > > > > > > already updated to the wiki KIP. You can take a look.
> > > > But
> > > > > > it's
> > > > > > > > > not
> > > > > > > > > > > the
> > > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > As for the implementation, the code is mostly done
> > > and
> > > > we
> > > > > > > > already
> > > > > > > > > > had
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > feature tests / system tests. I have added the
> > > > > performance
> > > > > > test
> > > > > > > > > > > results
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > the KIP. However the recent design changes (e.g.
> > > leader
> > > > > > epoch
> > > > > > > > > info
> > > > > > > > > > > > > > management / log truncation / some of the new
> > > metrics)
> > > > > > have not
> > > > > > > > > > been
> > > > > > > > > > > > > > implemented yet. It will take about 2 weeks for us to
> > > > > > implement
> > > > > > > > > > > after you
> > > > > > > > > > > > > > review and agree with those design changes.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun Rao <
> > > > jun@confluent.io
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Any new updates on the KIP? This feature is one of
> > > > the
> > > > > > most
> > > > > > > > > > > important
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > most requested features in Apache Kafka right now.
> > > It
> > > > > > would
> > > > > > > > be
> > > > > > > > > > > helpful
> > > > > > > > > > > > > if
> > > > > > > > > > > > > > > we can make sustained progress on this. Could you
> > > > share
> > > > > > how
> > > > > > > > far
> > > > > > > > > > > along
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > the design/implementation right now? Is there
> > > > anything
> > > > > > that
> > > > > > > > > other
> > > > > > > > > > > > > people
> > > > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As for "transactional support" and "follower
> > > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > > further comments from me as long as the producer
> > > > state
> > > > > > and
> > > > > > > > > leader
> > > > > > > > > > > epoch
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > be restored properly from the object store when
> > > > needed.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM Satish Duggana <
> > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > We did not want to add many implementation
> > > details
> > > > in
> > > > > > the
> > > > > > > > > KIP.
> > > > > > > > > > > But we
> > > > > > > > > > > > > > > > decided to add them in the KIP as appendix or
> > > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > > follower fetch protocol) to describe the flow
> > > with
> > > > > the
> > > > > > main
> > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > That will answer most of the queries. I will
> > > update
> > > > > on
> > > > > > this
> > > > > > > > > > mail
> > > > > > > > > > > > > > > > thread when the respective sections are updated.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM Alexandre Dupriez
> > > > > > > > > > > > > > > > <al...@gmail.com> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > A couple of questions specific to the section
> > > > > > "Follower
> > > > > > > > > > > > > > > > > Requests/Replication", pages 16:17 in the
> > > design
> > > > > > document
> > > > > > > > > > [1].
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900. It is mentioned that followers fetch
> > > > auxiliary
> > > > > > > > states
> > > > > > > > > > > from the
> > > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900.a Does the consistency model of the
> > > external
> > > > > > storage
> > > > > > > > > > > impacts
> > > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > > of leader epochs and other auxiliary data?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900.b What are the benefits of using a
> > > mechanism
> > > > to
> > > > > > store
> > > > > > > > > and
> > > > > > > > > > > > > access
> > > > > > > > > > > > > > > > > the leader epochs which is different from other
> > > > > > metadata
> > > > > > > > > > > associated
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > tiered segments? What are the benefits of
> > > > > retrieving
> > > > > > this
> > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > on-demand from the follower rather than relying
> > > > on
> > > > > > > > > > propagation
> > > > > > > > > > > via
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > topic __remote_log_metadata? What are the
> > > > > advantages
> > > > > > over
> > > > > > > > > > > using a
> > > > > > > > > > > > > > > > > dedicated control structure (e.g. a new record
> > > > > type)
> > > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > > this topic? Since in the document, different
> > > > > control
> > > > > > > > paths
> > > > > > > > > > are
> > > > > > > > > > > > > > > > > operating in the system, how are the metadata
> > > > > stored
> > > > > > in
> > > > > > > > > > > > > > > > > __remote_log_metadata [which also include the
> > > > epoch
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > which offloaded a segment] and the remote
> > > > auxiliary
> > > > > > > > states,
> > > > > > > > > > > kept in
> > > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900.c A follower can encounter an
> > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > > this in response to a Fetch or
> > > > OffsetForLeaderEpoch
> > > > > > > > > request?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900.d What happens if, after a follower
> > > > encountered
> > > > > > an
> > > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE response, its
> > > > > > attempts to
> > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > leader epochs fail (for instance, because the
> > > > > remote
> > > > > > > > > storage
> > > > > > > > > > is
> > > > > > > > > > > > > > > > > temporarily unavailable)? Does the follower
> > > > > > fallbacks to
> > > > > > > > a
> > > > > > > > > > mode
> > > > > > > > > > > > > where
> > > > > > > > > > > > > > > > > it ignores tiered segments, and applies
> > > > truncation
> > > > > > using
> > > > > > > > > only
> > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > available information? What happens when access
> > > > to
> > > > > > the
> > > > > > > > > remote
> > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > is restored? How is the replica lineage
> > > inferred
> > > > by
> > > > > > the
> > > > > > > > > > remote
> > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > epochs reconciled with the follower's replica
> > > > > > lineage,
> > > > > > > > > which
> > > > > > > > > > > has
> > > > > > > > > > > > > > > > > evolved? Does the follower remember fetching
> > > > > > auxiliary
> > > > > > > > > states
> > > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > > in the past and attempt reconciliation? Is
> > > there
> > > > a
> > > > > > plan
> > > > > > > > to
> > > > > > > > > > > offer
> > > > > > > > > > > > > > > > > different strategies in this scenario,
> > > > configurable
> > > > > > via
> > > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900.e Is the leader epoch cache offloaded with
> > > > > every
> > > > > > > > > segment?
> > > > > > > > > > > Or
> > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > a new checkpoint is detected? If that
> > > information
> > > > > is
> > > > > > not
> > > > > > > > > > always
> > > > > > > > > > > > > > > > > offloaded to avoid duplicating data, how does
> > > the
> > > > > > remote
> > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > satisfy the request to retrieve it?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900.f Since the leader epoch cache covers the
> > > > > entire
> > > > > > > > > replica
> > > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > > what happens if, after a leader epoch cache
> > > file
> > > > is
> > > > > > > > > offloaded
> > > > > > > > > > > with
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > given segment, the local epoch cache is
> > > truncated
> > > > > > [not
> > > > > > > > > > > necessarily
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > a range of offset included in tiered segments]?
> > > > How
> > > > > > are
> > > > > > > > > > remote
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > > local leader epoch caches kept consistent?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900.g Consumer can also use leader epochs (e.g.
> > > > to
> > > > > > enable
> > > > > > > > > > > fencing
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > protect against stale leaders). What
> > > differences
> > > > > > would
> > > > > > > > > there
> > > > > > > > > > be
> > > > > > > > > > > > > > > > > between consumer and follower fetches?
> > > > Especially,
> > > > > > would
> > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > > also fetch leader epoch information from the
> > > > remote
> > > > > > > > > storage?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 900.h Assume a newly elected leader of a
> > > > > > topic-partition
> > > > > > > > > > > detects
> > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > recent segments are available in the external
> > > > > > storage,
> > > > > > > > with
> > > > > > > > > > > epochs
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > its local epoch. Does it ignore these segments
> > > > and
> > > > > > their
> > > > > > > > > > > associated
> > > > > > > > > > > > > > > > > epoch-to-offset vectors? Or try to reconstruct
> > > > its
> > > > > > local
> > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > lineage based on the data remotely available?
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > > https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55, Satish Duggana <
> > > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > > Please let us know if you have any comments
> > > on
> > > > > > > > > > "transactional
> > > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > > and "follower requests/replication" mentioned
> > > > in
> > > > > > the
> > > > > > > > > wiki.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25 PM Satish
> > > Duggana <
> > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks Jun for your comments.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >100. It would be useful to provide more
> > > > > details
> > > > > > on
> > > > > > > > how
> > > > > > > > > > > those
> > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to really
> > > > > assess
> > > > > > > > > whether
> > > > > > > > > > > the
> > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > apis are sufficient/redundant. A few examples
> > > > below.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > We will update the wiki and let you know.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >100.1 deleteRecords seems to only advance
> > > > the
> > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > Log. How does that trigger the deletion of remote
> > > > log
> > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > RLMTask for leader partition periodically
> > > > > checks
> > > > > > > > > whether
> > > > > > > > > > > there
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > remote log segments earlier to
> > > logStartOffset
> > > > > > and the
> > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > remote log segment metadata and data are
> > > > > deleted
> > > > > > by
> > > > > > > > > using
> > > > > > > > > > > RLMM
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >100.2 stopReplica with deletion is used
> > > in 2
> > > > > > cases
> > > > > > > > (a)
> > > > > > > > > > > replica
> > > > > > > > > > > > > > > > reassignment; (b) topic deletion. We only want to
> > > > > > delete
> > > > > > > > the
> > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > metadata in the second case. Also, in the second
> > > > > case,
> > > > > > who
> > > > > > > > > > > initiates
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > deletion of the remote segment since the leader
> > > may
> > > > > not
> > > > > > > > > exist?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Right, it is deleted only incase of topic
> > > > > > deletion
> > > > > > > > > only.
> > > > > > > > > > We
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > cover
> > > > > > > > > > > > > > > > > > > the details in the KIP.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >100.3 "LogStartOffset of a topic can be
> > > > either
> > > > > > in
> > > > > > > > > local
> > > > > > > > > > > or in
> > > > > > > > > > > > > > > > remote storage." If LogStartOffset exists in both
> > > > > > places,
> > > > > > > > > which
> > > > > > > > > > > one
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > source of truth?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I meant the logStartOffset can point to
> > > > either
> > > > > of
> > > > > > > > local
> > > > > > > > > > > segment
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > remote segment but it is initialised and
> > > > > > maintained
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > Log
> > > > > > > > > > > > > > > class
> > > > > > > > > > > > > > > > > > > like now.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >100.4 List<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > topicPartition,
> > > > > > long
> > > > > > > > > > > minOffset):
> > > > > > > > > > > > > > How
> > > > > > > > > > > > > > > > is minOffset supposed to be used?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Returns list of remote segments, sorted by
> > > > > > baseOffset
> > > > > > > > > in
> > > > > > > > > > > > > > ascending
> > > > > > > > > > > > > > > > > > > order that have baseOffset >= the given min
> > > > > > Offset.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >100.5 When copying a segment to remote
> > > > > storage,
> > > > > > it
> > > > > > > > > seems
> > > > > > > > > > > we
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > calling the same RLMM.putRemoteLogSegmentData()
> > > > twice
> > > > > > > > before
> > > > > > > > > > and
> > > > > > > > > > > > > after
> > > > > > > > > > > > > > > > copyLogSegment(). Could you explain why?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > This is more about prepare/commit/rollback
> > > as
> > > > > you
> > > > > > > > > > > suggested.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > update the wiki with the new APIs.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >100.6 LogSegmentData includes
> > > > > leaderEpochCache,
> > > > > > but
> > > > > > > > > > there
> > > > > > > > > > > is
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > in RemoteStorageManager to retrieve it.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Nice catch, copy/paste issue. There is an
> > > API
> > > > > to
> > > > > > > > > retrieve
> > > > > > > > > > > it.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >101. If the __remote_log_metadata is for
> > > > > > production
> > > > > > > > > > usage,
> > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > you provide more details? For example, what is
> > > the
> > > > > > schema
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > data
> > > > > > > > > > > > > > > (both
> > > > > > > > > > > > > > > > key and value)? How is the topic
> > > maintained,delete
> > > > or
> > > > > > > > > compact?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > It is with delete config and it’s retention
> > > > > > period is
> > > > > > > > > > > suggested
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > more than the remote retention period.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >110. Is the cache implementation in
> > > > > > > > > > > RemoteLogMetadataManager
> > > > > > > > > > > > > > meant
> > > > > > > > > > > > > > > > for production usage? If so, could you provide
> > > more
> > > > > > details
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > and how/where the data is stored?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > The proposal is to have a cache (with
> > > default
> > > > > > > > > > > implementation
> > > > > > > > > > > > > > backed
> > > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > rocksdb) but it will be added in later
> > > > > versions.
> > > > > > We
> > > > > > > > > will
> > > > > > > > > > > add
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > future work items.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >111. "Committed offsets can be stored in a
> > > > > local
> > > > > > > > > file".
> > > > > > > > > > > Could
> > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > describe the format of the file and where it's
> > > > > stored?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > We will cover this in the KIP.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >112. Truncation of remote segments under
> > > > > unclean
> > > > > > > > > leader
> > > > > > > > > > > > > > election:
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > am not sure who figures out the truncated remote
> > > > > > segments
> > > > > > > > and
> > > > > > > > > > how
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > information is propagated to all replicas?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > We will add this in detail in the KIP.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >113. "If there are any failures in
> > > removing
> > > > > > remote
> > > > > > > > log
> > > > > > > > > > > > > segments
> > > > > > > > > > > > > > > > then those are stored in a specific topic
> > > (default
> > > > as
> > > > > > > > > > > > > > > > __remote_segments_to_be_deleted)". Is it
> > > necessary
> > > > to
> > > > > > add
> > > > > > > > yet
> > > > > > > > > > > another
> > > > > > > > > > > > > > > > internal topic? Could we just keep retrying?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > This is not really an internal topic, it
> > > will
> > > > > be
> > > > > > > > > exposed
> > > > > > > > > > > as a
> > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > configurable topic. After a few retries, we
> > > > > want
> > > > > > user
> > > > > > > > > to
> > > > > > > > > > > know
> > > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > > > > the failure so that they can take an action
> > > > > > later by
> > > > > > > > > > > consuming
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > this topic. We want to keep this simple
> > > > instead
> > > > > > of
> > > > > > > > > > retrying
> > > > > > > > > > > > > > > > > > > continuously and maintaining the deletion
> > > > state
> > > > > > etc.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >114. "We may not need to copy
> > > > > > producer-id-snapshot
> > > > > > > > as
> > > > > > > > > we
> > > > > > > > > > > are
> > > > > > > > > > > > > > > > copying only segments earlier to
> > > > last-stable-offset."
> > > > > > Hmm,
> > > > > > > > > not
> > > > > > > > > > > sure
> > > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > that. The producer snapshot includes things like
> > > > the
> > > > > > last
> > > > > > > > > > > timestamp
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > open producer id and can affect when those
> > > producer
> > > > > > ids are
> > > > > > > > > > > expired.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Sure, this will be added as part of the
> > > > > > > > LogSegmentData.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Fri, May 29, 2020 at 6:39 AM Jun Rao <
> > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Made another pass on the wiki. A few more
> > > > > > comments
> > > > > > > > > > below.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 100. It would be useful to provide more
> > > > > > details on
> > > > > > > > > how
> > > > > > > > > > > those
> > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to really
> > > > > assess
> > > > > > > > > whether
> > > > > > > > > > > the
> > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > apis are sufficient/redundant. A few examples
> > > > below.
> > > > > > > > > > > > > > > > > > > > 100.1 deleteRecords seems to only advance
> > > > the
> > > > > > > > > > > logStartOffset
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > Log. How does that trigger the deletion of remote
> > > > log
> > > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > > 100.2 stopReplica with deletion is used
> > > in
> > > > 2
> > > > > > cases
> > > > > > > > > (a)
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > reassignment; (b) topic deletion. We only want to
> > > > > > delete
> > > > > > > > the
> > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > metadata in the second case. Also, in the second
> > > > > case,
> > > > > > who
> > > > > > > > > > > initiates
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > deletion of the remote segment since the leader
> > > may
> > > > > not
> > > > > > > > > exist?
> > > > > > > > > > > > > > > > > > > > 100.3 "LogStartOffset of a topic can be
> > > > > either
> > > > > > in
> > > > > > > > > local
> > > > > > > > > > > or in
> > > > > > > > > > > > > > > > remote storage." If LogStartOffset exists in both
> > > > > > places,
> > > > > > > > > which
> > > > > > > > > > > one
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > source of truth?
> > > > > > > > > > > > > > > > > > > > 100.4 List<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > > topicPartition,
> > > > > > long
> > > > > > > > > > > minOffset):
> > > > > > > > > > > > > > How
> > > > > > > > > > > > > > > > is minOffset supposed to be used?
> > > > > > > > > > > > > > > > > > > > 100.5 When copying a segment to remote
> > > > > > storage, it
> > > > > > > > > > seems
> > > > > > > > > > > we
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > calling the same RLMM.putRemoteLogSegmentData()
> > > > twice
> > > > > > > > before
> > > > > > > > > > and
> > > > > > > > > > > > > after
> > > > > > > > > > > > > > > > copyLogSegment(). Could you explain why?
> > > > > > > > > > > > > > > > > > > > 100.6 LogSegmentData includes
> > > > > > leaderEpochCache, but
> > > > > > > > > > > there is
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > api in RemoteStorageManager to retrieve it.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 101. If the __remote_log_metadata is for
> > > > > > production
> > > > > > > > > > > usage,
> > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > you provide more details? For example, what is
> > > the
> > > > > > schema
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > data
> > > > > > > > > > > > > > > (both
> > > > > > > > > > > > > > > > key and value)? How is the topic
> > > maintained,delete
> > > > or
> > > > > > > > > compact?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 110. Is the cache implementation in
> > > > > > > > > > > RemoteLogMetadataManager
> > > > > > > > > > > > > > > meant
> > > > > > > > > > > > > > > > for production usage? If so, could you provide
> > > more
> > > > > > details
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > and how/where the data is stored?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 111. "Committed offsets can be stored in
> > > a
> > > > > > local
> > > > > > > > > file".
> > > > > > > > > > > Could
> > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > describe the format of the file and where it's
> > > > > stored?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 112. Truncation of remote segments under
> > > > > > unclean
> > > > > > > > > leader
> > > > > > > > > > > > > > election:
> > > > > > > > > > > > > > > > I am not sure who figures out the truncated
> > > remote
> > > > > > segments
> > > > > > > > > and
> > > > > > > > > > > how
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > information is propagated to all replicas?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 113. "If there are any failures in
> > > removing
> > > > > > remote
> > > > > > > > > log
> > > > > > > > > > > > > segments
> > > > > > > > > > > > > > > > then those are stored in a specific topic
> > > (default
> > > > as
> > > > > > > > > > > > > > > > __remote_segments_to_be_deleted)". Is it
> > > necessary
> > > > to
> > > > > > add
> > > > > > > > yet
> > > > > > > > > > > another
> > > > > > > > > > > > > > > > internal topic? Could we just keep retrying?
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 114. "We may not need to copy
> > > > > > producer-id-snapshot
> > > > > > > > as
> > > > > > > > > > we
> > > > > > > > > > > are
> > > > > > > > > > > > > > > > copying only segments earlier to
> > > > last-stable-offset."
> > > > > > Hmm,
> > > > > > > > > not
> > > > > > > > > > > sure
> > > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > that. The producer snapshot includes things like
> > > > the
> > > > > > last
> > > > > > > > > > > timestamp
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > open producer id and can affect when those
> > > producer
> > > > > > ids are
> > > > > > > > > > > expired.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Thu, May 28, 2020 at 5:38 AM Satish
> > > > > Duggana
> > > > > > <
> > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >> Hi Jun,
> > > > > > > > > > > > > > > > > > > >> Gentle reminder. Please go through the
> > > > > updated
> > > > > > > > wiki
> > > > > > > > > > and
> > > > > > > > > > > let
> > > > > > > > > > > > > us
> > > > > > > > > > > > > > > > know your comments.
> > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >> Thanks,
> > > > > > > > > > > > > > > > > > > >> Satish.
> > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >> On Tue, May 19, 2020 at 3:50 PM Satish
> > > > > > Duggana <
> > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > > >>> Hi Jun,
> > > > > > > > > > > > > > > > > > > >>> Please go through the wiki which has
> > > the
> > > > > > latest
> > > > > > > > > > > updates.
> > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > doc is updated frequently to be in sync with
> > > wiki.
> > > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > > >>> Thanks,
> > > > > > > > > > > > > > > > > > > >>> Satish.
> > > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > > >>> On Tue, May 19, 2020 at 12:30 AM Jun
> > > Rao
> > > > <
> > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > > >>>> Hi, Satish,
> > > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > > >>>> Thanks for the update. Just to
> > > clarify.
> > > > > > Which
> > > > > > > > doc
> > > > > > > > > > has
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > latest updates, the wiki or the google doc?
> > > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > > >>>> Jun
> > > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > > >>>> On Thu, May 14, 2020 at 10:38 AM
> > > Satish
> > > > > > Duggana
> > > > > > > > <
> > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > >>>>> Thanks for your comments.  We updated
> > > > the
> > > > > > KIP
> > > > > > > > > with
> > > > > > > > > > > more
> > > > > > > > > > > > > > > > details.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> >100. For each of the operations
> > > > related
> > > > > to
> > > > > > > > > > tiering,
> > > > > > > > > > > it
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > be useful to provide a description on how it
> > > works
> > > > > > with the
> > > > > > > > > new
> > > > > > > > > > > API.
> > > > > > > > > > > > > > > These
> > > > > > > > > > > > > > > > include things like consumer fetch, replica
> > > fetch,
> > > > > > > > > > > > > offsetForTimestamp,
> > > > > > > > > > > > > > > > retention (remote and local) by size, time and
> > > > > > > > > logStartOffset,
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > deletion, etc. This will tell us if the proposed
> > > > APIs
> > > > > > are
> > > > > > > > > > > sufficient.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> We addressed most of these APIs in
> > > the
> > > > > > KIP. We
> > > > > > > > > can
> > > > > > > > > > > add
> > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > details if needed.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> >101. For the default implementation
> > > > > based
> > > > > > on
> > > > > > > > > > > internal
> > > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > is it meant as a proof of concept or for
> > > production
> > > > > > usage?
> > > > > > > > I
> > > > > > > > > > > assume
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > it's the former. However, if it's the latter,
> > > then
> > > > > the
> > > > > > KIP
> > > > > > > > > > needs
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > describe the design in more detail.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> It is production usage as was
> > > mentioned
> > > > > in
> > > > > > an
> > > > > > > > > > earlier
> > > > > > > > > > > > > mail.
> > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > plan to update this section in the next few days.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> >102. When tiering a segment, the
> > > > segment
> > > > > > is
> > > > > > > > > first
> > > > > > > > > > > > > written
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the object store and then its metadata is written
> > > > to
> > > > > > RLMM
> > > > > > > > > using
> > > > > > > > > > > the
> > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > "void putRemoteLogSegmentData()". One potential
> > > > issue
> > > > > > with
> > > > > > > > > this
> > > > > > > > > > > > > > approach
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > that if the system fails after the first
> > > operation,
> > > > > it
> > > > > > > > > leaves a
> > > > > > > > > > > > > garbage
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the object store that's never reclaimed. One way
> > > to
> > > > > > improve
> > > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > two separate APIs, sth like
> > > > > > > > preparePutRemoteLogSegmentData()
> > > > > > > > > > and
> > > > > > > > > > > > > > > > commitPutRemoteLogSegmentData().
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> That is a good point. We currently
> > > > have a
> > > > > > > > > different
> > > > > > > > > > > way
> > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > > markers in the segment but your suggestion is
> > > much
> > > > > > better.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> >103. It seems that the transactional
> > > > > > support
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > > > > ability
> > > > > > > > > > > > > > > > to read from follower are missing.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> KIP is updated with transactional
> > > > > support,
> > > > > > > > > follower
> > > > > > > > > > > fetch
> > > > > > > > > > > > > > > > semantics, and reading from a follower.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> >104. It would be useful to provide a
> > > > > > testing
> > > > > > > > > plan
> > > > > > > > > > > for
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> We added a few tests by introducing
> > > > test
> > > > > > util
> > > > > > > > for
> > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > storage in the PR. We will provide the testing
> > > plan
> > > > > in
> > > > > > the
> > > > > > > > > next
> > > > > > > > > > > few
> > > > > > > > > > > > > > days.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> Thanks,
> > > > > > > > > > > > > > > > > > > >>>>> Satish.
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > > >>>>> On Wed, Feb 26, 2020 at 9:43 PM
> > > Harsha
> > > > > > > > > > Chintalapani <
> > > > > > > > > > > > > > > > kafka@harsha.io> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>> On Tue, Feb 25, 2020 at 12:46 PM,
> > > Jun
> > > > > Rao
> > > > > > <
> > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the updated doc. The new
> > > > API
> > > > > > seems
> > > > > > > > > to
> > > > > > > > > > > be an
> > > > > > > > > > > > > > > > improvement overall. A few more comments below.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 100. For each of the operations
> > > > related
> > > > > > to
> > > > > > > > > > > tiering, it
> > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > be useful to provide a description on how it
> > > works
> > > > > > with the
> > > > > > > > > new
> > > > > > > > > > > API.
> > > > > > > > > > > > > > > These
> > > > > > > > > > > > > > > > include things like consumer fetch, replica
> > > fetch,
> > > > > > > > > > > > > offsetForTimestamp,
> > > > > > > > > > > > > > > > retention
> > > > > > > > > > > > > > > > > > > >>>>>>> (remote and local) by size, time
> > > and
> > > > > > > > > > > logStartOffset,
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > deletion, etc. This will tell us if the proposed
> > > > APIs
> > > > > > are
> > > > > > > > > > > sufficient.
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>> Thanks for the feedback Jun. We will
> > > > add
> > > > > > more
> > > > > > > > > > > details
> > > > > > > > > > > > > > around
> > > > > > > > > > > > > > > > this.
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 101. For the default implementation
> > > > > > based on
> > > > > > > > > > > internal
> > > > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > is it meant as a proof of concept or for
> > > production
> > > > > > usage?
> > > > > > > > I
> > > > > > > > > > > assume
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > it's the former. However, if it's the latter,
> > > then
> > > > > the
> > > > > > KIP
> > > > > > > > > > needs
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > describe the design in more detail.
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>> Yes it meant to be for production
> > > use.
> > > > > > > > Ideally
> > > > > > > > > it
> > > > > > > > > > > would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > good to merge this in as the default
> > > implementation
> > > > > for
> > > > > > > > > > metadata
> > > > > > > > > > > > > > service.
> > > > > > > > > > > > > > > > We can add more details around design and
> > > testing.
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 102. When tiering a segment, the
> > > > > segment
> > > > > > is
> > > > > > > > > first
> > > > > > > > > > > > > written
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the object store and then its metadata is written
> > > > to
> > > > > > RLMM
> > > > > > > > > using
> > > > > > > > > > > the
> > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > "void putRemoteLogSegmentData()".
> > > > > > > > > > > > > > > > > > > >>>>>>> One potential issue with this
> > > > approach
> > > > > is
> > > > > > > > that
> > > > > > > > > if
> > > > > > > > > > > the
> > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > fails after the first operation, it leaves a
> > > > garbage
> > > > > > in the
> > > > > > > > > > > object
> > > > > > > > > > > > > > store
> > > > > > > > > > > > > > > > that's never reclaimed. One way to improve this
> > > is
> > > > to
> > > > > > have
> > > > > > > > > two
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > APIs, sth like preparePutRemoteLogSegmentData()
> > > and
> > > > > > > > > > > > > > > > commitPutRemoteLogSegmentData().
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 103. It seems that the
> > > transactional
> > > > > > support
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > > > > ability
> > > > > > > > > > > > > > > > to read from follower are missing.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 104. It would be useful to provide
> > > a
> > > > > > testing
> > > > > > > > > plan
> > > > > > > > > > > for
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>> We are working on adding more
> > > details
> > > > > > around
> > > > > > > > > > > > > transactional
> > > > > > > > > > > > > > > > support and coming up with test plan.
> > > > > > > > > > > > > > > > > > > >>>>>> Add system tests and integration
> > > > tests.
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Mon, Feb 24, 2020 at 8:10 AM
> > > > Satish
> > > > > > > > Duggana
> > > > > > > > > <
> > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > >>>>>>> Please look at the earlier reply
> > > and
> > > > > let
> > > > > > us
> > > > > > > > > know
> > > > > > > > > > > your
> > > > > > > > > > > > > > > > comments.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > >>>>>>> Satish.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Wed, Feb 12, 2020 at 4:06 PM
> > > > Satish
> > > > > > > > Duggana
> > > > > > > > > <
> > > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for your comments on the
> > > > > > separation of
> > > > > > > > > > > remote
> > > > > > > > > > > > > log
> > > > > > > > > > > > > > > > metadata storage and remote log storage.
> > > > > > > > > > > > > > > > > > > >>>>>>> We had a few discussions since
> > > early
> > > > > Jan
> > > > > > on
> > > > > > > > how
> > > > > > > > > > to
> > > > > > > > > > > > > > support
> > > > > > > > > > > > > > > > eventually consistent stores like S3 by
> > > uncoupling
> > > > > > remote
> > > > > > > > log
> > > > > > > > > > > segment
> > > > > > > > > > > > > > > > metadata and remote log storage. It is written
> > > with
> > > > > > details
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > doc
> > > > > > > > > > > > > > > > here(1). Below is the brief summary of the
> > > > discussion
> > > > > > from
> > > > > > > > > that
> > > > > > > > > > > doc.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> The current approach consists of
> > > > > pulling
> > > > > > the
> > > > > > > > > > > remote log
> > > > > > > > > > > > > > > > segment metadata from remote log storage APIs. It
> > > > > > worked
> > > > > > > > fine
> > > > > > > > > > for
> > > > > > > > > > > > > > > storages
> > > > > > > > > > > > > > > > like HDFS. But one of the problems of relying on
> > > > the
> > > > > > remote
> > > > > > > > > > > storage
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > maintain metadata is that tiered-storage needs to
> > > > be
> > > > > > > > strongly
> > > > > > > > > > > > > > consistent,
> > > > > > > > > > > > > > > > with an impact not only on the metadata(e.g. LIST
> > > > in
> > > > > > S3)
> > > > > > > > but
> > > > > > > > > > > also on
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > segment data(e.g. GET after a DELETE in S3). The
> > > > cost
> > > > > > of
> > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > metadata in remote storage needs to be factored
> > > in.
> > > > > > This is
> > > > > > > > > > true
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > case of S3, LIST APIs incur huge costs as you
> > > > raised
> > > > > > > > earlier.
> > > > > > > > > > > > > > > > > > > >>>>>>> So, it is good to separate the
> > > remote
> > > > > > storage
> > > > > > > > > > from
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > remote log metadata store. We refactored the
> > > > existing
> > > > > > > > > > > > > > > RemoteStorageManager
> > > > > > > > > > > > > > > > and introduced RemoteLogMetadataManager. Remote
> > > log
> > > > > > > > metadata
> > > > > > > > > > > store
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > give strong consistency semantics but remote log
> > > > > > storage
> > > > > > > > can
> > > > > > > > > be
> > > > > > > > > > > > > > > eventually
> > > > > > > > > > > > > > > > consistent.
> > > > > > > > > > > > > > > > > > > >>>>>>> We can have a default
> > > implementation
> > > > > for
> > > > > > > > > > > > > > > > RemoteLogMetadataManager which uses an internal
> > > > > > topic(as
> > > > > > > > > > > mentioned in
> > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > of our earlier emails) as storage. But users can
> > > > > always
> > > > > > > > > plugin
> > > > > > > > > > > their
> > > > > > > > > > > > > > own
> > > > > > > > > > > > > > > > RemoteLogMetadataManager implementation based on
> > > > > their
> > > > > > > > > > > environment.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Please go through the updated KIP
> > > and
> > > > > > let us
> > > > > > > > > know
> > > > > > > > > > > your
> > > > > > > > > > > > > > > > comments. We have started refactoring for the
> > > > changes
> > > > > > > > > mentioned
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > and there may be a few more updates to the APIs.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> [1]
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > > https://docs.google.com/document/d/1qfkBCWL1e7ZWkHU7brxKDBebq4ie9yK20XJnKbgAlew/edit?ts=5e208ec7#
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Dec 27, 2019 at 5:43 PM
> > > Ivan
> > > > > > > > Yurchenko
> > > > > > > > > <
> > > > > > > > > > > > > > > > ivan0yurchenko@gmail.com>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi all,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Jun:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (a) Cost: S3 list object requests
> > > > cost
> > > > > > $0.005
> > > > > > > > > per
> > > > > > > > > > > 1000
> > > > > > > > > > > > > > > > requests. If
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> have 100,000 partitions and want to
> > > > > pull
> > > > > > the
> > > > > > > > > > > metadata
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> partition
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> at
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the rate of 1/sec. It can cost
> > > > > $0.5/sec,
> > > > > > > > which
> > > > > > > > > is
> > > > > > > > > > > > > roughly
> > > > > > > > > > > > > > > > $40K per
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> day.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> I want to note here, that no
> > > > reasonably
> > > > > > > > durable
> > > > > > > > > > > storage
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > be cheap at 100k RPS. For example, DynamoDB might
> > > > > give
> > > > > > the
> > > > > > > > > same
> > > > > > > > > > > > > > ballpark
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> figures.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> If we want to keep the pull-based
> > > > > > approach,
> > > > > > > > we
> > > > > > > > > > can
> > > > > > > > > > > try
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > reduce this
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> number
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> in several ways: doing listings
> > > less
> > > > > > > > frequently
> > > > > > > > > > (as
> > > > > > > > > > > > > > Satish
> > > > > > > > > > > > > > > > mentioned, with the current defaults it's ~3.33k
> > > > RPS
> > > > > > for
> > > > > > > > your
> > > > > > > > > > > > > example),
> > > > > > > > > > > > > > > > batching listing operations in some way
> > > (depending
> > > > on
> > > > > > the
> > > > > > > > > > > storage; it
> > > > > > > > > > > > > > > might
> > > > > > > > > > > > > > > > require the change of RSM's interface).
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> There are different ways for doing
> > > > push
> > > > > > based
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > propagation.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Some
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> object stores may support that
> > > > already.
> > > > > > For
> > > > > > > > > > > example, S3
> > > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> events
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> notification
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> This sounds interesting. However, I
> > > > > see a
> > > > > > > > > couple
> > > > > > > > > > of
> > > > > > > > > > > > > > issues
> > > > > > > > > > > > > > > > using it:
> > > > > > > > > > > > > > > > > > > >>>>>>> 1. As I understand the
> > > documentation,
> > > > > > > > > > notification
> > > > > > > > > > > > > > delivery
> > > > > > > > > > > > > > > > is not guaranteed
> > > > > > > > > > > > > > > > > > > >>>>>>> and it's recommended to
> > > periodically
> > > > do
> > > > > > LIST
> > > > > > > > to
> > > > > > > > > > > fill
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > gaps. Which brings us back to the same LIST
> > > > > consistency
> > > > > > > > > > > guarantees
> > > > > > > > > > > > > > issue.
> > > > > > > > > > > > > > > > > > > >>>>>>> 2. The same goes for the broker
> > > > start:
> > > > > > to get
> > > > > > > > > the
> > > > > > > > > > > > > current
> > > > > > > > > > > > > > > > state, we
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> need
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to LIST.
> > > > > > > > > > > > > > > > > > > >>>>>>> 3. The dynamic set of multiple
> > > > > consumers
> > > > > > > > > (RSMs):
> > > > > > > > > > > AFAIK
> > > > > > > > > > > > > > SQS
> > > > > > > > > > > > > > > > and SNS
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> aren't
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> designed for such a case.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Alexandre:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> A.1 As commented on PR 7561, S3
> > > > > > consistency
> > > > > > > > > model
> > > > > > > > > > > > > [1][2]
> > > > > > > > > > > > > > > > implies RSM
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> cannot
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> relies solely on S3 APIs to
> > > guarantee
> > > > > the
> > > > > > > > > > expected
> > > > > > > > > > > > > strong
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> consistency. The
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> proposed implementation [3] would
> > > > need
> > > > > > to be
> > > > > > > > > > > updated to
> > > > > > > > > > > > > > > take
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> into
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> account. Let’s talk more about
> > > this.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thank you for the feedback. I
> > > clearly
> > > > > > see the
> > > > > > > > > > need
> > > > > > > > > > > for
> > > > > > > > > > > > > > > > changing the S3 implementation
> > > > > > > > > > > > > > > > > > > >>>>>>> to provide stronger consistency
> > > > > > guarantees.
> > > > > > > > As
> > > > > > > > > it
> > > > > > > > > > > see
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > this thread, there are
> > > > > > > > > > > > > > > > > > > >>>>>>> several possible approaches to
> > > this.
> > > > > > Let's
> > > > > > > > > > discuss
> > > > > > > > > > > > > > > > RemoteLogManager's contract and
> > > > > > > > > > > > > > > > > > > >>>>>>> behavior (like pull vs push model)
> > > > > > further
> > > > > > > > > before
> > > > > > > > > > > > > picking
> > > > > > > > > > > > > > > > one (or
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> several -
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> ?) of them.
> > > > > > > > > > > > > > > > > > > >>>>>>> I'm going to do some evaluation of
> > > > > > DynamoDB
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > > > > > > pull-based
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> approach,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> if it's possible to apply it
> > > paying a
> > > > > > > > > reasonable
> > > > > > > > > > > bill.
> > > > > > > > > > > > > > > Also,
> > > > > > > > > > > > > > > > of the push-based approach
> > > > > > > > > > > > > > > > > > > >>>>>>> with a Kafka topic as the medium.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> A.2.3 Atomicity – what does an
> > > > > > implementation
> > > > > > > > > of
> > > > > > > > > > > RSM
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> respect to atomicity of the APIs
> > > > > > > > > copyLogSegment,
> > > > > > > > > > > > > > > > cleanupLogUntil and deleteTopicPartition? If a
> > > > > partial
> > > > > > > > > failure
> > > > > > > > > > > > > happens
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > any of those
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (e.g.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the S3 implementation, if one of
> > > the
> > > > > > multiple
> > > > > > > > > > > uploads
> > > > > > > > > > > > > > fails
> > > > > > > > > > > > > > > > [4]),
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> The S3 implementation is going to
> > > > > > change, but
> > > > > > > > > > it's
> > > > > > > > > > > > > worth
> > > > > > > > > > > > > > > > clarifying
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> anyway.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> The segment log file is being
> > > > uploaded
> > > > > > after
> > > > > > > > S3
> > > > > > > > > > has
> > > > > > > > > > > > > acked
> > > > > > > > > > > > > > > > uploading of all other files associated with the
> > > > > > segment
> > > > > > > > and
> > > > > > > > > > only
> > > > > > > > > > > > > after
> > > > > > > > > > > > > > > > this the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> whole
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> segment file set becomes visible
> > > > > > remotely for
> > > > > > > > > > > > > operations
> > > > > > > > > > > > > > > > like listRemoteSegments [1].
> > > > > > > > > > > > > > > > > > > >>>>>>> In case of upload failure, the
> > > files
> > > > > > that has
> > > > > > > > > > been
> > > > > > > > > > > > > > > > successfully
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> uploaded
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> stays
> > > > > > > > > > > > > > > > > > > >>>>>>> as invisible garbage that is
> > > > collected
> > > > > by
> > > > > > > > > > > > > cleanupLogUntil
> > > > > > > > > > > > > > > (or
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> overwritten
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> successfully later).
> > > > > > > > > > > > > > > > > > > >>>>>>> And the opposite happens during the
> > > > > > deletion:
> > > > > > > > > log
> > > > > > > > > > > files
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > deleted
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> first.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> This approach should generally work
> > > > > when
> > > > > > we
> > > > > > > > > solve
> > > > > > > > > > > > > > > > consistency issues by adding a strongly
> > > consistent
> > > > > > > > storage: a
> > > > > > > > > > > > > segment's
> > > > > > > > > > > > > > > > uploaded files
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> remain
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> invisible garbage until some
> > > metadata
> > > > > > about
> > > > > > > > > them
> > > > > > > > > > is
> > > > > > > > > > > > > > > written.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> A.3 Caching – storing locally the
> > > > > > segments
> > > > > > > > > > > retrieved
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > the remote storage is excluded as it does not
> > > align
> > > > > > with
> > > > > > > > the
> > > > > > > > > > > original
> > > > > > > > > > > > > > > intent
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> and even
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> defeat some of its purposes (save
> > > > disk
> > > > > > space
> > > > > > > > > > etc.).
> > > > > > > > > > > > > That
> > > > > > > > > > > > > > > > said, could
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> there
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> be other types of use cases where
> > > the
> > > > > > pattern
> > > > > > > > > of
> > > > > > > > > > > access
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> remotely
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> stored segments would benefit from
> > > > > local
> > > > > > > > > caching
> > > > > > > > > > > (and
> > > > > > > > > > > > > > > > potentially read-ahead)? Consider the use case
> > > of a
> > > > > > large
> > > > > > > > > pool
> > > > > > > > > > of
> > > > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> start
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> a backfill at the same time for one
> > > > day
> > > > > > worth
> > > > > > > > > of
> > > > > > > > > > > data
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > one year
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> ago
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> stored remotely. Caching the
> > > segments
> > > > > > locally
> > > > > > > > > > would
> > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> uncouple the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> load on the remote storage from the
> > > > > load
> > > > > > on
> > > > > > > > the
> > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > cluster. Maybe
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> RLM could expose a configuration
> > > > > > parameter to
> > > > > > > > > > > switch
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > feature
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> on/off?
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> I tend to agree here, caching
> > > remote
> > > > > > segments
> > > > > > > > > > > locally
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > making this configurable sounds pretty practical
> > > to
> > > > > > me. We
> > > > > > > > > > should
> > > > > > > > > > > > > > > implement
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> this,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> maybe not in the first iteration.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Br,
> > > > > > > > > > > > > > > > > > > >>>>>>> Ivan
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> [1]
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > > https://github.com/harshach/kafka/pull/18/files#diff-4d73d01c16caed6f2548fc3063550ef0R152
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Thu, 19 Dec 2019 at 19:49,
> > > > Alexandre
> > > > > > > > > Dupriez <
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> alexandre.dupriez@gmail.com>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thank you for the feedback. I am
> > > > trying
> > > > > > to
> > > > > > > > > > > understand
> > > > > > > > > > > > > > how a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> push-based
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> approach would work.
> > > > > > > > > > > > > > > > > > > >>>>>>> In order for the metadata to be
> > > > > > propagated
> > > > > > > > > (under
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > assumption you stated), would you plan to add a
> > > new
> > > > > > API in
> > > > > > > > > > Kafka
> > > > > > > > > > > to
> > > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > > the metadata store to send them directly to the
> > > > > > brokers?
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > >>>>>>> Alexandre
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Le mer. 18 déc. 2019 à 20:14, Jun
> > > > Rao <
> > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > écrit :
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Ivan,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> There are different ways for doing
> > > > push
> > > > > > based
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> propagation. Some
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> object stores may support that
> > > > already.
> > > > > > For
> > > > > > > > > > > example, S3
> > > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> events
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> notification (
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > >
> > > > > >
> > > https://docs.aws.amazon.com/AmazonS3/latest/dev/NotificationHowTo.html
> > > > > > > > > > > > > > ).
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Otherwise one could use a separate
> > > > > > metadata
> > > > > > > > > store
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> push-based
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> change propagation. Other people
> > > have
> > > > > > > > mentioned
> > > > > > > > > > > using a
> > > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> topic. The
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> best approach may depend on the
> > > > object
> > > > > > store
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > operational environment (e.g. whether an external
> > > > > > metadata
> > > > > > > > > > store
> > > > > > > > > > > is
> > > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> available).
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> The above discussion is based on
> > > the
> > > > > > > > assumption
> > > > > > > > > > > that we
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> cache the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> object metadata locally in every
> > > > > broker.
> > > > > > I
> > > > > > > > > > > mentioned
> > > > > > > > > > > > > > > earlier
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> an
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> alternative is to just
> > > store/retrieve
> > > > > > those
> > > > > > > > > > > metadata in
> > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > external metadata store. That may simplify the
> > > > > > > > implementation
> > > > > > > > > > in
> > > > > > > > > > > some
> > > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Dec 5, 2019 at 7:01 AM
> > > Satish
> > > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for your reply.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Currently, `listRemoteSegments` is
> > > > > > called at
> > > > > > > > > the
> > > > > > > > > > > > > > configured
> > > > > > > > > > > > > > > > interval(not every second, defaults to 30secs).
> > > > > Storing
> > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> log
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> metadata in a strongly consistent
> > > > store
> > > > > > for
> > > > > > > > S3
> > > > > > > > > > RSM
> > > > > > > > > > > is
> > > > > > > > > > > > > > > raised
> > > > > > > > > > > > > > > > in PR-comment[1].
> > > > > > > > > > > > > > > > > > > >>>>>>> RLM invokes RSM at regular
> > > intervals
> > > > > and
> > > > > > RSM
> > > > > > > > > can
> > > > > > > > > > > give
> > > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> segment
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> metadata if it is available. RSM is
> > > > > > > > responsible
> > > > > > > > > > for
> > > > > > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> fetching those entries. It should
> > > be
> > > > > > based on
> > > > > > > > > > > whatever
> > > > > > > > > > > > > > > > mechanism
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> consistent and efficient with the
> > > > > > respective
> > > > > > > > > > remote
> > > > > > > > > > > > > > > storage.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Can you give more details about
> > > push
> > > > > > based
> > > > > > > > > > > mechanism
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > RSM?
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 1.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > https://github.com/apache/kafka/pull/7561#discussion_r344576223
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > >>>>>>> Satish.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Dec 5, 2019 at 4:23 AM Jun
> > > > Rao
> > > > > <
> > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Harsha,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 40/41. I am curious which block
> > > > > storages
> > > > > > you
> > > > > > > > > have
> > > > > > > > > > > > > tested.
> > > > > > > > > > > > > > > S3
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> seems
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to be
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> one of the popular block stores.
> > > The
> > > > > > concerns
> > > > > > > > > > that
> > > > > > > > > > > I
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> pull
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> based
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> approach are the following.
> > > > > > > > > > > > > > > > > > > >>>>>>> (a) Cost: S3 list object requests
> > > > cost
> > > > > > $0.005
> > > > > > > > > per
> > > > > > > > > > > 1000
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> requests. If
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> have 100,000 partitions and want to
> > > > > pull
> > > > > > the
> > > > > > > > > > > metadata
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> partition
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> at
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the rate of 1/sec. It can cost
> > > > > $0.5/sec,
> > > > > > > > which
> > > > > > > > > is
> > > > > > > > > > > > > roughly
> > > > > > > > > > > > > > > > $40K
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> per
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> day.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (b) Semantics: S3 list objects are
> > > > > > eventually
> > > > > > > > > > > > > consistent.
> > > > > > > > > > > > > > > So,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> when
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> do a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> list object request, there is no
> > > > > > guarantee
> > > > > > > > that
> > > > > > > > > > > you can
> > > > > > > > > > > > > > see
> > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> uploaded
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> objects. This could impact the
> > > > > > correctness of
> > > > > > > > > > > > > subsequent
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> logics.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (c) Efficiency: Blindly pulling
> > > > > metadata
> > > > > > when
> > > > > > > > > > > there is
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> change adds
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> unnecessary overhead in the broker
> > > as
> > > > > > well as
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > block
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> store.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> So, have you guys tested S3? If so,
> > > > > > could you
> > > > > > > > > > share
> > > > > > > > > > > > > your
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> experience
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> terms of cost, semantics and
> > > > > efficiency?
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Dec 3, 2019 at 10:11 PM
> > > > Harsha
> > > > > > > > > > > Chintalapani <
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> kafka@harsha.io
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 26, 2019 at 3:46 PM,
> > > Jun
> > > > > Rao
> > > > > > <
> > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Ying,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 40/41. There are two different ways
> > > > > that
> > > > > > we
> > > > > > > > can
> > > > > > > > > > > > > approach
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> this.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> One is
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> what
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> you said. We can have an
> > > opinionated
> > > > > way
> > > > > > of
> > > > > > > > > > > storing and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> populating
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> metadata that we think is good
> > > enough
> > > > > for
> > > > > > > > > > > everyone. I
> > > > > > > > > > > > > am
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> sure if
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> is the case based on what's
> > > currently
> > > > > > > > proposed
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> For
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> example, I
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> am not sure that (1) everyone
> > > always
> > > > > > needs
> > > > > > > > > local
> > > > > > > > > > > > > > metadata;
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (2)
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> current
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> local storage format is general
> > > > enough
> > > > > > and
> > > > > > > > (3)
> > > > > > > > > > > everyone
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wants to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> use
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> pull based approach to propagate
> > > the
> > > > > > > > metadata.
> > > > > > > > > > > Another
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> approach
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> is to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> make
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> this pluggable and let the
> > > > implementor
> > > > > > > > > implements
> > > > > > > > > > > the
> > > > > > > > > > > > > > best
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> approach
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> for a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> particular block storage. I haven't
> > > > > seen
> > > > > > any
> > > > > > > > > > > comments
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Slack/AirBnb
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the mailing list on this topic. It
> > > > > would
> > > > > > be
> > > > > > > > > great
> > > > > > > > > > > if
> > > > > > > > > > > > > they
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> provide
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> feedback directly here.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> The current interfaces are designed
> > > > > with
> > > > > > most
> > > > > > > > > > > popular
> > > > > > > > > > > > > > block
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storages
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> available today and we did 2
> > > > > > implementations
> > > > > > > > > with
> > > > > > > > > > > these
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> interfaces and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> they both are yielding good results
> > > > as
> > > > > we
> > > > > > > > going
> > > > > > > > > > > through
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> testing of
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> it.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> If there is ever a need for pull
> > > > based
> > > > > > > > approach
> > > > > > > > > > we
> > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> definitely
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> evolve
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the interface.
> > > > > > > > > > > > > > > > > > > >>>>>>> In the past we did mark interfaces
> > > to
> > > > > be
> > > > > > > > > evolving
> > > > > > > > > > > to
> > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> room for
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> unknowns
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> in the future.
> > > > > > > > > > > > > > > > > > > >>>>>>> If you have any suggestions around
> > > > the
> > > > > > > > current
> > > > > > > > > > > > > interfaces
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> please
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> propose we
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> are happy to see if we can work
> > > them
> > > > > > into it.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 43. To offer tier storage as a
> > > > general
> > > > > > > > feature,
> > > > > > > > > > > ideally
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> existing
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> capabilities should still be
> > > > supported.
> > > > > > It's
> > > > > > > > > fine
> > > > > > > > > > > if
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> uber
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> implementation doesn't support all
> > > > > > > > capabilities
> > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> internal
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> usage.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> However, the framework should be
> > > > > general
> > > > > > > > > enough.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> We agree on that as a principle.
> > > But
> > > > > all
> > > > > > of
> > > > > > > > > these
> > > > > > > > > > > major
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> features
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> mostly
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> coming right now and to have a new
> > > > big
> > > > > > > > feature
> > > > > > > > > > > such as
> > > > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> support all the new features will
> > > be
> > > > a
> > > > > > big
> > > > > > > > ask.
> > > > > > > > > > We
> > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> document on
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> how
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> do
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> we approach solving these in future
> > > > > > > > iterations.
> > > > > > > > > > > > > > > > > > > >>>>>>> Our goal is to make this tiered
> > > > storage
> > > > > > > > feature
> > > > > > > > > > > work
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> everyone.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 43.3 This is more than just serving
> > > > the
> > > > > > > > tier-ed
> > > > > > > > > > > data
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> block
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> With KIP-392, the consumer now can
> > > > > > resolve
> > > > > > > > the
> > > > > > > > > > > > > conflicts
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> with the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> replica
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> based on leader epoch. So, we need
> > > to
> > > > > > make
> > > > > > > > sure
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> leader epoch
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> can be
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> recovered properly from tier
> > > storage.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> We are working on testing our
> > > > approach
> > > > > > and we
> > > > > > > > > > will
> > > > > > > > > > > > > update
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the KIP
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> design details.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 43.4 For JBOD, if tier storage
> > > stores
> > > > > the
> > > > > > > > tier
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> locally, we
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> need to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> support moving such metadata across
> > > > > disk
> > > > > > > > > > > directories
> > > > > > > > > > > > > > since
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> JBOD
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> supports
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> moving data across disks.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> KIP is updated with JBOD details.
> > > > > Having
> > > > > > said
> > > > > > > > > > that
> > > > > > > > > > > JBOD
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> tooling
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> needs
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> evolve to support production loads.
> > > > > Most
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > users
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> interested in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> using tiered storage without JBOD
> > > > > support
> > > > > > > > > support
> > > > > > > > > > > on
> > > > > > > > > > > > > day
> > > > > > > > > > > > > > 1.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > >>>>>>> Harsha
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> As for meeting, we could have a KIP
> > > > > > e-meeting
> > > > > > > > > on
> > > > > > > > > > > this
> > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> needed,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> but it
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> will be open to everyone and will
> > > be
> > > > > > recorded
> > > > > > > > > and
> > > > > > > > > > > > > shared.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Often,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> details are still resolved through
> > > > the
> > > > > > > > mailing
> > > > > > > > > > > list.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 19, 2019 at 6:48 PM
> > > Ying
> > > > > > Zheng
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> <yi...@uber.com.invalid>
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Please ignore my previous email
> > > > > > > > > > > > > > > > > > > >>>>>>> I didn't know Apache requires all
> > > the
> > > > > > > > > discussions
> > > > > > > > > > > to be
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> "open"
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 19, 2019, 5:40 PM Ying
> > > > > Zheng
> > > > > > <
> > > > > > > > > > > > > yingz@uber.com
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thank you very much for your
> > > > feedback!
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Can we schedule a meeting in your
> > > > Palo
> > > > > > Alto
> > > > > > > > > > office
> > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> December? I
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> think a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> face to face discussion is much
> > > more
> > > > > > > > efficient
> > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> emails. Both
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Harsha
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> I can visit you. Satish may be able
> > > > to
> > > > > > join
> > > > > > > > us
> > > > > > > > > > > > > remotely.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Nov 15, 2019 at 11:04 AM
> > > Jun
> > > > > Rao
> > > > > > <
> > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Harsha,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> The following is a more detailed
> > > high
> > > > > > level
> > > > > > > > > > > feedback
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the KIP.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Overall,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the KIP seems useful. The challenge
> > > > is
> > > > > > how to
> > > > > > > > > > > design it
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> such that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> it’s
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> general enough to support different
> > > > > ways
> > > > > > of
> > > > > > > > > > > > > implementing
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> feature
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> support existing features.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 40. Local segment metadata storage:
> > > > The
> > > > > > KIP
> > > > > > > > > makes
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> assumption
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> metadata for the archived log
> > > > segments
> > > > > > are
> > > > > > > > > cached
> > > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> every
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> broker
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> and provides a specific
> > > > implementation
> > > > > > for
> > > > > > > > the
> > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> framework. We probably should
> > > discuss
> > > > > > this
> > > > > > > > > more.
> > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> example,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> some
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage providers may not want to
> > > > cache
> > > > > > the
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> locally and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> just
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> rely
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> upon a remote key/value store if
> > > > such a
> > > > > > store
> > > > > > > > > is
> > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> present. If
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> local store is used, there could be
> > > > > > different
> > > > > > > > > > ways
> > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> implementing it
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (e.g., based on customized local
> > > > files,
> > > > > > an
> > > > > > > > > > embedded
> > > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> store
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> like
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> RocksDB, etc). An alternative of
> > > > > > designing
> > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> provide an
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> interface for retrieving the tier
> > > > > segment
> > > > > > > > > > metadata
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> leave the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> details
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> of
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> how to get the metadata outside of
> > > > the
> > > > > > > > > framework.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 41. RemoteStorageManager interface
> > > > and
> > > > > > the
> > > > > > > > > usage
> > > > > > > > > > > of the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> interface in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> framework: I am not sure if the
> > > > > > interface is
> > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> enough. For
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> example,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> it seems that RemoteLogIndexEntry
> > > is
> > > > > > tied to
> > > > > > > > a
> > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> way of
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storing
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> metadata in remote storage. The
> > > > > framework
> > > > > > > > uses
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> listRemoteSegments()
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> api
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> a pull based approach. However, in
> > > > some
> > > > > > other
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> implementations, a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> push
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> based
> > > > > > > > > > > > > > > > > > > >>>>>>> approach may be more preferred. I
> > > > don’t
> > > > > > have
> > > > > > > > a
> > > > > > > > > > > concrete
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> proposal
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> yet.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> But,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> it would be useful to give this
> > > area
> > > > > some
> > > > > > > > more
> > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> and see
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> if we
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> make the interface more general.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 42. In the diagram, the
> > > > > RemoteLogManager
> > > > > > is
> > > > > > > > > side
> > > > > > > > > > by
> > > > > > > > > > > > > side
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> LogManager.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> This KIP only discussed how the
> > > fetch
> > > > > > request
> > > > > > > > > is
> > > > > > > > > > > > > handled
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> between
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> two
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> layer. However, we should also
> > > > consider
> > > > > > how
> > > > > > > > > other
> > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> touch
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> log can be handled. e.g., list
> > > > offsets
> > > > > by
> > > > > > > > > > > timestamp,
> > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> records,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> etc.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Also, in this model, it's not clear
> > > > > which
> > > > > > > > > > > component is
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> responsible
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> for
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> managing the log start offset. It
> > > > seems
> > > > > > that
> > > > > > > > > the
> > > > > > > > > > > log
> > > > > > > > > > > > > > start
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> offset
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> could
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> be
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> changed by both RemoteLogManager
> > > and
> > > > > > > > > LogManager.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 43. There are quite a few existing
> > > > > > features
> > > > > > > > not
> > > > > > > > > > > covered
> > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> KIP.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> It
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> would be useful to discuss each of
> > > > > those.
> > > > > > > > > > > > > > > > > > > >>>>>>> 43.1 I won’t say that compacted
> > > > topics
> > > > > > are
> > > > > > > > > rarely
> > > > > > > > > > > used
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> always
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> small.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> For example, KStreams uses
> > > compacted
> > > > > > topics
> > > > > > > > for
> > > > > > > > > > > storing
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> states
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> sometimes the size of the topic
> > > could
> > > > > be
> > > > > > > > large.
> > > > > > > > > > > While
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> might
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> be ok
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> support compacted topics initially,
> > > > it
> > > > > > would
> > > > > > > > be
> > > > > > > > > > > useful
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> have a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> high
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> level
> > > > > > > > > > > > > > > > > > > >>>>>>> idea on how this might be supported
> > > > > down
> > > > > > the
> > > > > > > > > road
> > > > > > > > > > > so
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> we
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> don’t
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> have
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> make incompatible API changes in
> > > the
> > > > > > future.
> > > > > > > > > > > > > > > > > > > >>>>>>> 43.2 We need to discuss how EOS is
> > > > > > supported.
> > > > > > > > > In
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> particular, how
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> producer state integrated with the
> > > > > remote
> > > > > > > > > > storage.
> > > > > > > > > > > 43.3
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Now that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> KIP-392
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (allow consumers to fetch from
> > > > closest
> > > > > > > > replica)
> > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> implemented,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> we
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> need
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> discuss how reading from a follower
> > > > > > replica
> > > > > > > > is
> > > > > > > > > > > > > supported
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> 43.4 We need to discuss how JBOD is
> > > > > > supported
> > > > > > > > > > with
> > > > > > > > > > > tier
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Nov 8, 2019 at 12:06 AM Tom
> > > > > > Bentley <
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> tbentley@redhat.com
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks for those insights Ying.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Nov 7, 2019 at 9:26 PM Ying
> > > > > Zheng
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> <yingz@uber.com.invalid
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Thanks, I missed that point.
> > > However,
> > > > > > there's
> > > > > > > > > > > still a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> point at
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> which
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> consumer fetches start getting
> > > served
> > > > > > from
> > > > > > > > > remote
> > > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (even
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> if
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> point isn't as soon as the local
> > > log
> > > > > > > > retention
> > > > > > > > > > > > > > time/size).
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> This
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> represents
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> a kind of performance cliff edge
> > > and
> > > > > > what I'm
> > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> interested
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> how
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> easy it is for a consumer which
> > > falls
> > > > > off
> > > > > > > > that
> > > > > > > > > > > cliff to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> catch up
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> and so
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> its
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> fetches again come from local
> > > > storage.
> > > > > > > > > Obviously
> > > > > > > > > > > this
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> depend
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> on
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> all
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> sorts of factors (like production
> > > > rate,
> > > > > > > > > > consumption
> > > > > > > > > > > > > > rate),
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> so
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> it's
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> guaranteed (just like it's not
> > > > > > guaranteed for
> > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> today), but
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> would
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> represent a new failure mode.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> As I have explained in the last
> > > mail,
> > > > > > it's a
> > > > > > > > > very
> > > > > > > > > > > rare
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> case that
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> a
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> consumer
> > > > > > > > > > > > > > > > > > > >>>>>>> need to read remote data. With our
> > > > > > experience
> > > > > > > > > at
> > > > > > > > > > > Uber,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> this only
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> happens
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> when the consumer service had an
> > > > outage
> > > > > > for
> > > > > > > > > > several
> > > > > > > > > > > > > > hours.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> There is not a "performance cliff"
> > > as
> > > > > you
> > > > > > > > > assume.
> > > > > > > > > > > The
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> even faster than local disks in
> > > terms
> > > > > of
> > > > > > > > > > bandwidth.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Reading from
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage is going to have higher
> > > > latency
> > > > > > than
> > > > > > > > > > local
> > > > > > > > > > > > > disk.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> But
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> since
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> consumer
> > > > > > > > > > > > > > > > > > > >>>>>>> is catching up several hours data,
> > > > it's
> > > > > > not
> > > > > > > > > > > sensitive
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> sub-second
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> level
> > > > > > > > > > > > > > > > > > > >>>>>>> latency, and each remote read
> > > request
> > > > > > will
> > > > > > > > > read a
> > > > > > > > > > > large
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> amount of
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> data to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> make the overall performance better
> > > > > than
> > > > > > > > > reading
> > > > > > > > > > > from
> > > > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> disks.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Another aspect I'd like to
> > > understand
> > > > > > better
> > > > > > > > is
> > > > > > > > > > the
> > > > > > > > > > > > > > effect
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> of
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> serving
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> fetch
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> request from remote storage has on
> > > > the
> > > > > > > > broker's
> > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> utilization. If
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> we're just trimming the amount of
> > > > data
> > > > > > held
> > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> (without
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> increasing
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> overall local+remote retention),
> > > then
> > > > > > we're
> > > > > > > > > > > effectively
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> trading
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> disk
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> bandwidth for network bandwidth
> > > when
> > > > > > serving
> > > > > > > > > > fetch
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> requests from
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage (which I understand to be a
> > > > > good
> > > > > > > > thing,
> > > > > > > > > > > since
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> brokers are
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> often/usually disk bound). But if
> > > > we're
> > > > > > > > > > increasing
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> overall
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> local+remote
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> retention then it's more likely
> > > that
> > > > > > network
> > > > > > > > > > itself
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> becomes the
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> bottleneck.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> I appreciate this is all rather
> > > hand
> > > > > > wavy,
> > > > > > > > I'm
> > > > > > > > > > just
> > > > > > > > > > > > > > trying
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> understand
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> how this would affect broker
> > > > > > performance, so
> > > > > > > > > I'd
> > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> grateful for
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> any
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> insights you can offer.
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> Network bandwidth is a function of
> > > > > > produce
> > > > > > > > > speed,
> > > > > > > > > > > it
> > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> nothing
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> do
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> remote retention. As long as the
> > > data
> > > > > is
> > > > > > > > > shipped
> > > > > > > > > > to
> > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> storage,
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> keep the data there for 1 day or 1
> > > > year
> > > > > > or
> > > > > > > > 100
> > > > > > > > > > > years,
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> doesn't
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> consume
> > > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>> any
> > > > > > > > > > > > > > > > > > > >>>>>>> network resources.
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >


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

Posted by Satish Duggana <sa...@gmail.com>.
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%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition


Satish.


On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng <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 <ju...@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 <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
> > > unclean-leader-election.
> > > 2) Since the user set local retention time (or local retention bytes), I
> > > think we are expected to
> > > keep that much local data when possible (avoid truncating all the local
> > > data). But, as you said,
> > > unclean leader elections are very rare, this may not be a big problem.
> > >
> > > The current design uses the leader broker as source-of-truth. This is
> > > consistent with the
> > > existing Kafka behavior.
> > >
> > > By using remote storage as the source-of-truth, the follower logic can
> > be a
> > > little simpler,
> > > but the leader logic is going to be more complex. Overall, I don't see
> > > there many benefits
> > > of using remote storage as the source-of-truth.
> > >
> > >
> > >
> > > On Tue, Jul 28, 2020 at 10:25 AM Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Hi, Satish,
> > > >
> > > > Thanks for the reply.
> > > >
> > > > 1001. In your example, I was thinking that you could just download the
> > > > latest leader epoch from the object store. After that you know the
> > leader
> > > > should end with offset 1100. The leader will delete all its local data
> > > > before offset 1000 and start accepting new messages at offset 1100.
> > > > Consumer requests for messages before offset 1100 will be served from
> > the
> > > > object store. The benefit with this approach is that it's simpler to
> > > reason
> > > > about who is the source of truth. The downside is slightly  increased
> > > > unavailability window during unclean leader election. Since unclean
> > > leader
> > > > elections are rare, I am not sure if this is a big concern.
> > > >
> > > > 1008. Yes, I think introducing sth like local.retention.ms seems more
> > > > consistent.
> > > >
> > > > Jun
> > > >
> > > > On Tue, Jul 28, 2020 at 2:30 AM Satish Duggana <
> > satish.duggana@gmail.com
> > > >
> > > > wrote:
> > > >
> > > > > HI Jun,
> > > > > Thanks for your comments. We put our inline replies below.
> > > > >
> > > > > 1001. I was thinking that you could just use the tiered metadata to
> > do
> > > > the
> > > > > reconciliation. The tiered metadata contains offset ranges and epoch
> > > > > history. Those should be enough for reconciliation purposes.
> > > > >
> > > > > If we use remote storage as the source-of-truth during
> > > > > unclean-leader-election, it's possible that after reconciliation the
> > > > > remote storage will have more recent data than the new leader's local
> > > > > storage. For example, the new leader's latest message is offset 1000,
> > > > > while the remote storage has message 1100. In such a case, the new
> > > > > leader will have to download the messages from 1001 to 1100, before
> > > > > accepting new messages from producers. Otherwise, there would be a
> > gap
> > > > > in the local data between 1000 and 1101.
> > > > >
> > > > > Moreover, with the current design, leader epoch history is stored in
> > > > > remote storage, rather than the metadata topic. We did consider
> > saving
> > > > > epoch history in remote segment metadata. But the concern is that
> > > > > there is currently no limit for the epoch history size.
> > Theoretically,
> > > > > if a user has a very long remote retention time and there are very
> > > > > frequent leadership changes, the leader epoch history can become too
> > > > > long to fit into a regular Kafka message.
> > > > >
> > > > >
> > > > > 1003.3 Having just a serverEndpoint string is probably not enough.
> > > > > Connecting to a Kafka cluster may need various security credentials.
> > We
> > > > can
> > > > > make RLMM configurable and pass in the properties through the
> > > configure()
> > > > > method. Ditto for RSM.
> > > > >
> > > > > RLMM and  RSM are already configurable and they take properties which
> > > > > start with "remote.log.metadata." and "remote.log.storage."
> > > > > respectively and a few others. We have listener-name as the config
> > for
> > > > > RLMM and other properties(like security) can be sent as you
> > suggested.
> > > > > We will update the KIP with the details.
> > > > >
> > > > >
> > > > > 1008.1 We started with log.retention.hours and log.retention.minutes,
> > > and
> > > > > added log.retention.ms later. If we are adding a new configuration,
> > ms
> > > > > level config alone is enough and is simpler. We can build tools to
> > make
> > > > the
> > > > > configuration at different granularities easier. The definition of
> > > > > log.retention.ms is "The number of milliseconds to keep a log file
> > > > before
> > > > > deleting it". The deletion is independent of whether tiering is
> > enabled
> > > > or
> > > > > not. If this changes to just the local portion of the data, we are
> > > > changing
> > > > > the meaning of an existing configuration.
> > > > >
> > > > > We are fine with either way. We can go with log.retention.xxxx as the
> > > > > effective log retention instead of local log retention. With this
> > > > > convention, we need to introduce  local.log.retention instead of
> > > > > remote.log.retention.ms that we proposed. If log.retention.ms as -1
> > > > > then remote retention is also considered as unlimited but user should
> > > > > be able to set the local.retention.ms.
> > > > > So, we need to introduce local.log.retention.ms and
> > > > > local.log.retention.bytes which should  always  be <=
> > > > > log.retention.ms/bytes respectively.
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Jul 24, 2020 at 3:37 AM Jun Rao <ju...@confluent.io> wrote:
> > > > > >
> > > > > > Hi, Satish,
> > > > > >
> > > > > > Thanks for the reply. A few quick comments below.
> > > > > >
> > > > > > 1001. I was thinking that you could just use the tiered metadata to
> > > do
> > > > > the
> > > > > > reconciliation. The tiered metadata contains offset ranges and
> > epoch
> > > > > > history. Those should be enough for reconciliation purposes.
> > > > > >
> > > > > > 1003.3 Having just a serverEndpoint string is probably not enough.
> > > > > > Connecting to a Kafka cluster may need various security
> > credentials.
> > > We
> > > > > can
> > > > > > make RLMM configurable and pass in the properties through the
> > > > configure()
> > > > > > method. Ditto for RSM.
> > > > > >
> > > > > > 1008.1 We started with log.retention.hours and
> > log.retention.minutes,
> > > > and
> > > > > > added log.retention.ms later. If we are adding a new
> > configuration,
> > > ms
> > > > > > level config alone is enough and is simpler. We can build tools to
> > > make
> > > > > the
> > > > > > configuration at different granularities easier. The definition of
> > > > > > log.retention.ms is "The number of milliseconds to keep a log file
> > > > > before
> > > > > > deleting it". The deletion is independent of whether tiering is
> > > enabled
> > > > > or
> > > > > > not. If this changes to just the local portion of the data, we are
> > > > > changing
> > > > > > the meaning of an existing configuration.
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Thu, Jul 23, 2020 at 11:04 AM Satish Duggana <
> > > > > satish.duggana@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Jun,
> > > > > > >
> > > > > > > Thank you for the comments! Ying, Harsha and I discussed and put
> > > our
> > > > > > > comments below.
> > > > > > >
> > > > > > >
> > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > elections.
> > > > > This
> > > > > > > is very useful, but I am wondering if this is the best approach.
> > My
> > > > > > > understanding of the proposed approach is to allow the new
> > > (unclean)
> > > > > leader
> > > > > > > to take new messages immediately. While this increases
> > > availability,
> > > > it
> > > > > > > creates the problem that there could be multiple conflicting
> > > segments
> > > > > in
> > > > > > > the remote store for the same offset range. This seems to make it
> > > > > harder
> > > > > > > for RLMM to determine which archived log segments contain the
> > > correct
> > > > > data.
> > > > > > > For example, an archived log segment could at one time be the
> > > correct
> > > > > data,
> > > > > > > but be changed to incorrect data after an unclean leader
> > election.
> > > An
> > > > > > > alternative approach is to let the unclean leader use the
> > archived
> > > > > data as
> > > > > > > the source of truth. So, when the new (unclean) leader takes
> > over,
> > > it
> > > > > first
> > > > > > > reconciles the local data based on the archived data before
> > taking
> > > > new
> > > > > > > messages. This makes the job of RLMM a bit easier since all
> > > archived
> > > > > data
> > > > > > > are considered correct. This increases availability a bit.
> > However,
> > > > > since
> > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > >
> > > > > > > Firstly, We don't want to assume the remote storage is more
> > > reliable
> > > > > than
> > > > > > > Kafka. Kafka unclean leader election usually happens when there
> > is
> > > a
> > > > > large
> > > > > > > scale outage that impacts multiple racks (or even multiple
> > > > availability
> > > > > > > zones). In such a case, the remote storage may be unavailable or
> > > > > unstable.
> > > > > > > Pulling a large amount of data from the remote storage to
> > reconcile
> > > > the
> > > > > > > local data may also exacerbate the outage. With the current
> > design,
> > > > > the new
> > > > > > > leader can start working even when the remote storage is
> > > temporarily
> > > > > > > unavailable.
> > > > > > >
> > > > > > > Secondly, it is not easier to implement the reconciling logic at
> > > the
> > > > > leader
> > > > > > > side. It can take a long time for the new leader to download the
> > > > remote
> > > > > > > data and rebuild local producer id / leader epoch information.
> > > During
> > > > > this
> > > > > > > period, the leader cannot accept any requests from the clients
> > and
> > > > > > > followers. We have to introduce a new state for the leader, and a
> > > new
> > > > > error
> > > > > > > code to let the clients / followers know what is happening.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > 1002. RemoteStorageManager.
> > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > RemoteStorageManager.
> > > > > We
> > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all other
> > methods,
> > > > we
> > > > > pass
> > > > > > > in RemoteLogSegmentMetadata.
> > > > > > >
> > > > > > > Nice catch, we can have the RemoteLogSegmentMetadata for
> > > > copyLogSegment
> > > > > > > too.
> > > > > > >
> > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata inclusive or
> > > > exclusive?
> > > > > > >
> > > > > > > It is inclusive.
> > > > > > >
> > > > > > > 1002.3 It seems that we need an api to get the leaderEpoch
> > history
> > > > for
> > > > > a
> > > > > > > partition.
> > > > > > >
> > > > > > > Yes, updated the KIP with the new method.
> > > > > > >
> > > > > > >
> > > > > > > 1002.4 Could you define the type of RemoteLogSegmentContext?
> > > > > > >
> > > > > > > This is removed in the latest code and it is not needed.
> > > > > > >
> > > > > > >
> > > > > > > 1003 RemoteLogMetadataManager
> > > > > > >
> > > > > > > 1003.1 I am not sure why we need both of the following methods
> > > > > > > in RemoteLogMetadataManager. Could we combine them into one that
> > > > takes
> > > > > in
> > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > >     RemoteLogSegmentId getRemoteLogSegmentId(TopicPartition
> > > > > topicPartition,
> > > > > > > long offset) throws IOException;
> > > > > > >     RemoteLogSegmentMetadata
> > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > >
> > > > > > > Good point, these can be merged for now. I guess we needed them
> > in
> > > > > earlier
> > > > > > > version of the implementation but it is not needed now.
> > > > > > >
> > > > > > > 1003.2 There seems to be some inconsistencies in the methods
> > > below. I
> > > > > am
> > > > > > > not sure why one takes RemoteLogSegmentMetadata and the other
> > > > > > > takes RemoteLogSegmentId.
> > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > >     void deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > >
> > > > > > > RLMM stores RemoteLogSegmentMetadata which is identified by
> > > > > > > RemoteLogsSegmentId. So, when it is added it takes
> > > > > > > RemoteLogSegmentMetadata. `delete` operation needs only
> > > > > RemoteLogsSegmentId
> > > > > > > as RemoteLogSegmentMetadata can be identified with
> > > > RemoteLogsSegmentId.
> > > > > > >
> > > > > > > 1003.3 In void onServerStarted(final String serverEndpoint), what
> > > > > > > is serverEndpoint used for?
> > > > > > >
> > > > > > > This can be used by RLMM implementation to connect to the local
> > > Kafka
> > > > > > > cluster. Incase of default  implementation, it is used in
> > > > initializing
> > > > > > > kafka clients connecting to the local cluster.
> > > > > > >
> > > > > > > 1004. It would be useful to document how all the new APIs are
> > being
> > > > > used.
> > > > > > > For example, when is RemoteLogSegmentMetadata.markedForDeletion
> > > being
> > > > > set
> > > > > > > and used? How are
> > > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset being
> > > > used?
> > > > > > >
> > > > > > > RLMM APIs are going through the changes and they should be ready
> > > in a
> > > > > few
> > > > > > > days. I will update the KIP and the mail  thread once they are
> > > ready.
> > > > > > >
> > > > > > > 1005. Handling partition deletion: The KIP says "RLMM will
> > > eventually
> > > > > > > delete these segments by using RemoteStorageManager." Which
> > replica
> > > > > does
> > > > > > > this logic?
> > > > > > >
> > > > > > > This is a good point. When a topic is deleted, it will not have
> > any
> > > > > > > leader/followers to do the cleanup. We will have a cleaner agent
> > > on a
> > > > > > > single broker in the cluster to do this cleanup, we plan to add
> > > that
> > > > in
> > > > > > > controller broker.
> > > > > > >
> > > > > > > 1006. "If there are any failures in removing remote log segments
> > > then
> > > > > those
> > > > > > > are stored in a specific topic (default as
> > > > > __remote_segments_to_be_deleted)
> > > > > > > and user can consume the events(which contain
> > > remote-log-segment-id)
> > > > > from
> > > > > > > that topic and clean them up from remote storage.  " Not sure if
> > > it's
> > > > > worth
> > > > > > > the complexity of adding another topic. Could we just retry?
> > > > > > >
> > > > > > > Sure, we can keep this simpler for now by logging an error after
> > > > > retries.
> > > > > > > We can give users a better way to process this in future. Oneway
> > > can
> > > > > be a
> > > > > > > dead letter topic which can be configured by the user.
> > > > > > >
> > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the existing
> > > > > > > fetchPurgatory?
> > > > > > >
> > > > > > > We have 2 types of delayed operations waiting for 2 different
> > > events.
> > > > > > > DelayedFetch waits for new messages from producers.
> > > > DelayedRemoteFetch
> > > > > > > waits for the remote-storage-read-task to finish. When either of
> > > the
> > > > 2
> > > > > > > events happens, we only want to notify one type of the delayed
> > > > > operations.
> > > > > > > It would be inefficient to put 2 types of delayed operations in
> > one
> > > > > > > purgatory, as the tryComplete() methods of the delayed operations
> > > can
> > > > > be
> > > > > > > triggered by irrelevant events.
> > > > > > >
> > > > > > >
> > > > > > > 1008. Configurations:
> > > > > > > 1008.1 remote.log.retention.ms, remote.log.retention.minutes,
> > > > > > > remote.log.retention.hours: It seems that we just need the ms
> > one.
> > > > > Also,
> > > > > > > are we changing the meaning of existing config log.retention.ms
> > to
> > > > > mean
> > > > > > > the
> > > > > > > local retention? For backward compatibility, it's better to not
> > > > change
> > > > > the
> > > > > > > meaning of existing configurations.
> > > > > > >
> > > > > > > We agree that we only need remote.log.retention.ms. But, the
> > > > existing
> > > > > > > Kafka
> > > > > > > configuration
> > > > > > > has 3 properties (log.retention.ms, log.retention.minutes,
> > > > > > > log.retention.hours). We just
> > > > > > > want to keep consistent with the existing properties.
> > > > > > > Existing log.retention.xxxx config is about log retention in
> > > broker’s
> > > > > > > storage which is local. It should be easy for users to configure
> > > > > partition
> > > > > > > storage with local retention and remote retention based on their
> > > > usage.
> > > > > > >
> > > > > > > 1008.2 Should remote.log.storage.enable be at the topic level?
> > > > > > >
> > > > > > > We can introduce topic level config for the same remote.log
> > > settings.
> > > > > User
> > > > > > > can set the desired config while creating the topic.
> > > > > > > remote.log.storage.enable property is not allowed to be updated
> > > after
> > > > > the
> > > > > > > topic is created. Other remote.log.* properties can be modified.
> > We
> > > > > will
> > > > > > > support flipping remote.log.storage.enable in next versions.
> > > > > > >
> > > > > > > 1009. It would be useful to list all limitations in a separate
> > > > section:
> > > > > > > compacted topic, JBOD, etc. Also, is changing a topic from delete
> > > to
> > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > >
> > > > > > > +1 to have limitations in a separate section. We will update the
> > > KIP
> > > > > with
> > > > > > > that.
> > > > > > > Topic  created with effective value for remote.log.enabled as
> > true,
> > > > > can not
> > > > > > > change its retention policy from delete to compact.
> > > > > > >
> > > > > > > 1010. Thanks for performance numbers. Are those with RocksDB as
> > the
> > > > > cache?
> > > > > > >
> > > > > > > No, We have not yet added RocksDB support. This is based on
> > > in-memory
> > > > > map
> > > > > > > representation. We will add that support and update this thread
> > > after
> > > > > > > updating the KIP with the numbers.
> > > > > > >
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Jul 21, 2020 at 6:49 AM Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Hi, Satish, Ying, Harsha,
> > > > > > > >
> > > > > > > > Thanks for the updated KIP. A few more comments below.
> > > > > > > >
> > > > > > > > 1000. Regarding Colin's question on querying the metadata
> > > directly
> > > > > in the
> > > > > > > > remote block store. One issue is that not all block stores
> > offer
> > > > the
> > > > > > > needed
> > > > > > > > api to query the metadata. For example, S3 only offers an api
> > to
> > > > list
> > > > > > > > objects under a prefix and this api has the eventual
> > consistency
> > > > > > > semantic.
> > > > > > > >
> > > > > > > > 1001. The KIP described a few scenarios of unclean leader
> > > > elections.
> > > > > This
> > > > > > > > is very useful, but I am wondering if this is the best
> > approach.
> > > My
> > > > > > > > understanding of the proposed approach is to allow the new
> > > > (unclean)
> > > > > > > leader
> > > > > > > > to take new messages immediately. While this increases
> > > > availability,
> > > > > it
> > > > > > > > creates the problem that there could be multiple conflicting
> > > > > segments in
> > > > > > > > the remote store for the same offset range. This seems to make
> > it
> > > > > harder
> > > > > > > > for RLMM to determine which archived log segments contain the
> > > > correct
> > > > > > > data.
> > > > > > > > For example, an archived log segment could at one time be the
> > > > correct
> > > > > > > data,
> > > > > > > > but be changed to incorrect data after an unclean leader
> > > election.
> > > > An
> > > > > > > > alternative approach is to let the unclean leader use the
> > > archived
> > > > > data
> > > > > > > as
> > > > > > > > the source of truth. So, when the new (unclean) leader takes
> > > over,
> > > > it
> > > > > > > first
> > > > > > > > reconciles the local data based on the archived data before
> > > taking
> > > > > new
> > > > > > > > messages. This makes the job of RLMM a bit easier since all
> > > > archived
> > > > > data
> > > > > > > > are considered correct. This increases availability a bit.
> > > However,
> > > > > since
> > > > > > > > unclean leader elections are rare, this may be ok.
> > > > > > > >
> > > > > > > > 1002. RemoteStorageManager.
> > > > > > > > 1002.1 There seems to be some inconsistencies in
> > > > > RemoteStorageManager. We
> > > > > > > > pass in RemoteLogSegmentId copyLogSegment(). For all other
> > > methods,
> > > > > we
> > > > > > > pass
> > > > > > > > in RemoteLogSegmentMetadata.
> > > > > > > > 1002.2 Is endOffset in RemoteLogSegmentMetadata inclusive or
> > > > > exclusive?
> > > > > > > > 1002.3 It seems that we need an api to get the leaderEpoch
> > > history
> > > > > for a
> > > > > > > > partition.
> > > > > > > > 1002.4 Could you define the type of RemoteLogSegmentContext?
> > > > > > > >
> > > > > > > > 1003 RemoteLogMetadataManager
> > > > > > > > 1003.1 I am not sure why we need both of the following methods
> > > > > > > > in RemoteLogMetadataManager. Could we combine them into one
> > that
> > > > > takes in
> > > > > > > > offset and returns RemoteLogSegmentMetadata?
> > > > > > > >     RemoteLogSegmentId getRemoteLogSegmentId(TopicPartition
> > > > > > > topicPartition,
> > > > > > > > long offset) throws IOException;
> > > > > > > >     RemoteLogSegmentMetadata
> > > > > > > getRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > 1003.2 There seems to be some inconsistencies in the methods
> > > below.
> > > > > I am
> > > > > > > > not sure why one takes RemoteLogSegmentMetadata and the other
> > > > > > > > takes RemoteLogSegmentId.
> > > > > > > >     void putRemoteLogSegmentData(RemoteLogSegmentMetadata
> > > > > > > > remoteLogSegmentMetadata) throws IOException;
> > > > > > > >     void deleteRemoteLogSegmentMetadata(RemoteLogSegmentId
> > > > > > > > remoteLogSegmentId) throws IOException;
> > > > > > > > 1003.3 In void onServerStarted(final String serverEndpoint),
> > what
> > > > > > > > is serverEndpoint used for?
> > > > > > > >
> > > > > > > > 1004. It would be useful to document how all the new APIs are
> > > being
> > > > > used.
> > > > > > > > For example, when is RemoteLogSegmentMetadata.markedForDeletion
> > > > > being set
> > > > > > > > and used? How are
> > > > > > > > RemoteLogMetadataManager.earliestLogOffset/highestLogOffset
> > being
> > > > > used?
> > > > > > > >
> > > > > > > > 1005. Handling partition deletion: The KIP says "RLMM will
> > > > eventually
> > > > > > > > delete these segments by using RemoteStorageManager." Which
> > > replica
> > > > > does
> > > > > > > > this logic?
> > > > > > > >
> > > > > > > > 1006. "If there are any failures in removing remote log
> > segments
> > > > then
> > > > > > > those
> > > > > > > > are stored in a specific topic (default as
> > > > > > > __remote_segments_to_be_deleted)
> > > > > > > > and user can consume the events(which contain
> > > > remote-log-segment-id)
> > > > > from
> > > > > > > > that topic and clean them up from remote storage.  " Not sure
> > if
> > > > it's
> > > > > > > worth
> > > > > > > > the complexity of adding another topic. Could we just retry?
> > > > > > > >
> > > > > > > > 1007. RemoteFetchPurgatory: Could we just reuse the existing
> > > > > > > > fetchPurgatory?
> > > > > > > >
> > > > > > > > 1008. Configurations:
> > > > > > > > 1008.1 remote.log.retention.ms, remote.log.retention.minutes,
> > > > > > > > remote.log.retention.hours: It seems that we just need the ms
> > > one.
> > > > > Also,
> > > > > > > > are we changing the meaning of existing config
> > log.retention.ms
> > > to
> > > > > mean
> > > > > > > > the
> > > > > > > > local retention? For backward compatibility, it's better to not
> > > > > change
> > > > > > > the
> > > > > > > > meaning of existing configurations.
> > > > > > > > 1008.2 Should remote.log.storage.enable be at the topic level?
> > > > > > > >
> > > > > > > > 1009. It would be useful to list all limitations in a separate
> > > > > section:
> > > > > > > > compacted topic, JBOD, etc. Also, is changing a topic from
> > delete
> > > > to
> > > > > > > > compact and vice versa allowed when tiering is enabled?
> > > > > > > >
> > > > > > > > 1010. Thanks for performance numbers. Are those with RocksDB as
> > > the
> > > > > > > cache?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Wed, Jul 15, 2020 at 6:12 PM Harsha Ch <harsha.ch@gmail.com
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Colin,
> > > > > > > > >                Thats not what we said in the previous email.
> > > RLMM
> > > > > is
> > > > > > > > > pluggable storage and by running numbers even 1PB data you do
> > > not
> > > > > need
> > > > > > > > more
> > > > > > > > > than 10GB local storage.
> > > > > > > > > If in future this becomes a blocker for any users we can
> > > revisit
> > > > > but
> > > > > > > this
> > > > > > > > > does not warrant another implementation at this point to push
> > > the
> > > > > data
> > > > > > > to
> > > > > > > > > remote storage.
> > > > > > > > > We can ofcourse implement another RLMM that is optional for
> > > users
> > > > > to
> > > > > > > > > configure to push to remote. But that doesn't need to be
> > > > addressed
> > > > > in
> > > > > > > > this
> > > > > > > > > KIP.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Harsha
> > > > > > > > >
> > > > > > > > > On Wed, Jul 15, 2020 at 5:50 PM Colin McCabe <
> > > cmccabe@apache.org
> > > > >
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Ying,
> > > > > > > > > >
> > > > > > > > > > Thanks for the response.
> > > > > > > > > >
> > > > > > > > > > It sounds like you agree that storing the metadata in the
> > > > remote
> > > > > > > > storage
> > > > > > > > > > would be a better design overall.  Given that that's true,
> > is
> > > > > there
> > > > > > > any
> > > > > > > > > > reason to include the worse implementation based on
> > RocksDB?
> > > > > > > > > >
> > > > > > > > > > Choosing a long-term metadata store is not something that
> > we
> > > > > should
> > > > > > > do
> > > > > > > > > > lightly.  It can take users years to migrate from metadata
> > > > store
> > > > > to
> > > > > > > the
> > > > > > > > > > other.  I also don't think it's realistic or desirable for
> > > > users
> > > > > to
> > > > > > > > write
> > > > > > > > > > their own metadata stores.  Even assuming that they could
> > do
> > > a
> > > > > good
> > > > > > > job
> > > > > > > > > at
> > > > > > > > > > this, it would create huge fragmentation in the Kafka
> > > > ecosystem.
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Jul 14, 2020, at 09:39, Ying Zheng wrote:
> > > > > > > > > > > Hi Jun,
> > > > > > > > > > > Hi Colin,
> > > > > > > > > > >
> > > > > > > > > > > Satish and I are still discussing some details about how
> > to
> > > > > handle
> > > > > > > > > > > transactions / producer ids. Satish is going to make some
> > > > minor
> > > > > > > > changes
> > > > > > > > > > to
> > > > > > > > > > > RLMM API and other parts. Other than that, we have
> > finished
> > > > > > > updating
> > > > > > > > > the
> > > > > > > > > > KIP
> > > > > > > > > > >
> > > > > > > > > > > I agree with Colin that the current design of using
> > rocksDB
> > > > is
> > > > > not
> > > > > > > > > > > optimal. But this design is simple and should work for
> > > almost
> > > > > all
> > > > > > > the
> > > > > > > > > > > existing Kafka users. RLMM is a plugin. Users can replace
> > > > > rocksDB
> > > > > > > > with
> > > > > > > > > > > their own RLMM implementation, if needed. So, I think we
> > > can
> > > > > keep
> > > > > > > > > rocksDB
> > > > > > > > > > > for now. What do you think?
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Ying
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Jul 7, 2020 at 10:35 AM Jun Rao <
> > jun@confluent.io>
> > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi, Ying,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the update. It's good to see the progress on
> > > > this.
> > > > > > > > Please
> > > > > > > > > > let us
> > > > > > > > > > > > know when you are done updating the KIP wiki.
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng
> > > > > > > <yingz@uber.com.invalid
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Satish and I have added more design details in the
> > KIP,
> > > > > > > including
> > > > > > > > > > how to
> > > > > > > > > > > > > keep consistency between replicas (especially when
> > > there
> > > > is
> > > > > > > > > > leadership
> > > > > > > > > > > > > changes / log truncations) and new metrics. We also
> > > made
> > > > > some
> > > > > > > > other
> > > > > > > > > > minor
> > > > > > > > > > > > > changes in the doc. We will finish the KIP changes in
> > > the
> > > > > next
> > > > > > > > > > couple of
> > > > > > > > > > > > > days. We will let you know when we are done. Most of
> > > the
> > > > > > > changes
> > > > > > > > > are
> > > > > > > > > > > > > already updated to the wiki KIP. You can take a look.
> > > But
> > > > > it's
> > > > > > > > not
> > > > > > > > > > the
> > > > > > > > > > > > > final version yet.
> > > > > > > > > > > > >
> > > > > > > > > > > > > As for the implementation, the code is mostly done
> > and
> > > we
> > > > > > > already
> > > > > > > > > had
> > > > > > > > > > > > some
> > > > > > > > > > > > > feature tests / system tests. I have added the
> > > > performance
> > > > > test
> > > > > > > > > > results
> > > > > > > > > > > > in
> > > > > > > > > > > > > the KIP. However the recent design changes (e.g.
> > leader
> > > > > epoch
> > > > > > > > info
> > > > > > > > > > > > > management / log truncation / some of the new
> > metrics)
> > > > > have not
> > > > > > > > > been
> > > > > > > > > > > > > implemented yet. It will take about 2 weeks for us to
> > > > > implement
> > > > > > > > > > after you
> > > > > > > > > > > > > review and agree with those design changes.
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Jul 7, 2020 at 9:23 AM Jun Rao <
> > > jun@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi, Satish, Harsha,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Any new updates on the KIP? This feature is one of
> > > the
> > > > > most
> > > > > > > > > > important
> > > > > > > > > > > > and
> > > > > > > > > > > > > > most requested features in Apache Kafka right now.
> > It
> > > > > would
> > > > > > > be
> > > > > > > > > > helpful
> > > > > > > > > > > > if
> > > > > > > > > > > > > > we can make sustained progress on this. Could you
> > > share
> > > > > how
> > > > > > > far
> > > > > > > > > > along
> > > > > > > > > > > > is
> > > > > > > > > > > > > > the design/implementation right now? Is there
> > > anything
> > > > > that
> > > > > > > > other
> > > > > > > > > > > > people
> > > > > > > > > > > > > > can help to get it across the line?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > As for "transactional support" and "follower
> > > > > > > > > > requests/replication", no
> > > > > > > > > > > > > > further comments from me as long as the producer
> > > state
> > > > > and
> > > > > > > > leader
> > > > > > > > > > epoch
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > be restored properly from the object store when
> > > needed.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Jun 9, 2020 at 3:39 AM Satish Duggana <
> > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > We did not want to add many implementation
> > details
> > > in
> > > > > the
> > > > > > > > KIP.
> > > > > > > > > > But we
> > > > > > > > > > > > > > > decided to add them in the KIP as appendix or
> > > > > > > > > > sub-sections(including
> > > > > > > > > > > > > > > follower fetch protocol) to describe the flow
> > with
> > > > the
> > > > > main
> > > > > > > > > > cases.
> > > > > > > > > > > > > > > That will answer most of the queries. I will
> > update
> > > > on
> > > > > this
> > > > > > > > > mail
> > > > > > > > > > > > > > > thread when the respective sections are updated.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Sat, Jun 6, 2020 at 7:49 PM Alexandre Dupriez
> > > > > > > > > > > > > > > <al...@gmail.com> wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Hi Satish,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > A couple of questions specific to the section
> > > > > "Follower
> > > > > > > > > > > > > > > > Requests/Replication", pages 16:17 in the
> > design
> > > > > document
> > > > > > > > > [1].
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900. It is mentioned that followers fetch
> > > auxiliary
> > > > > > > states
> > > > > > > > > > from the
> > > > > > > > > > > > > > > > remote storage.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900.a Does the consistency model of the
> > external
> > > > > storage
> > > > > > > > > > impacts
> > > > > > > > > > > > > reads
> > > > > > > > > > > > > > > > of leader epochs and other auxiliary data?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900.b What are the benefits of using a
> > mechanism
> > > to
> > > > > store
> > > > > > > > and
> > > > > > > > > > > > access
> > > > > > > > > > > > > > > > the leader epochs which is different from other
> > > > > metadata
> > > > > > > > > > associated
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > tiered segments? What are the benefits of
> > > > retrieving
> > > > > this
> > > > > > > > > > > > information
> > > > > > > > > > > > > > > > on-demand from the follower rather than relying
> > > on
> > > > > > > > > propagation
> > > > > > > > > > via
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > topic __remote_log_metadata? What are the
> > > > advantages
> > > > > over
> > > > > > > > > > using a
> > > > > > > > > > > > > > > > dedicated control structure (e.g. a new record
> > > > type)
> > > > > > > > > > propagated via
> > > > > > > > > > > > > > > > this topic? Since in the document, different
> > > > control
> > > > > > > paths
> > > > > > > > > are
> > > > > > > > > > > > > > > > operating in the system, how are the metadata
> > > > stored
> > > > > in
> > > > > > > > > > > > > > > > __remote_log_metadata [which also include the
> > > epoch
> > > > > of
> > > > > > > the
> > > > > > > > > > leader
> > > > > > > > > > > > > > > > which offloaded a segment] and the remote
> > > auxiliary
> > > > > > > states,
> > > > > > > > > > kept in
> > > > > > > > > > > > > > > > sync?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900.c A follower can encounter an
> > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE.
> > > > > > > > > > > > Is
> > > > > > > > > > > > > > > > this in response to a Fetch or
> > > OffsetForLeaderEpoch
> > > > > > > > request?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900.d What happens if, after a follower
> > > encountered
> > > > > an
> > > > > > > > > > > > > > > > OFFSET_MOVED_TO_TIERED_STORAGE response, its
> > > > > attempts to
> > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > leader epochs fail (for instance, because the
> > > > remote
> > > > > > > > storage
> > > > > > > > > is
> > > > > > > > > > > > > > > > temporarily unavailable)? Does the follower
> > > > > fallbacks to
> > > > > > > a
> > > > > > > > > mode
> > > > > > > > > > > > where
> > > > > > > > > > > > > > > > it ignores tiered segments, and applies
> > > truncation
> > > > > using
> > > > > > > > only
> > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > available information? What happens when access
> > > to
> > > > > the
> > > > > > > > remote
> > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > is restored? How is the replica lineage
> > inferred
> > > by
> > > > > the
> > > > > > > > > remote
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > epochs reconciled with the follower's replica
> > > > > lineage,
> > > > > > > > which
> > > > > > > > > > has
> > > > > > > > > > > > > > > > evolved? Does the follower remember fetching
> > > > > auxiliary
> > > > > > > > states
> > > > > > > > > > > > failed
> > > > > > > > > > > > > > > > in the past and attempt reconciliation? Is
> > there
> > > a
> > > > > plan
> > > > > > > to
> > > > > > > > > > offer
> > > > > > > > > > > > > > > > different strategies in this scenario,
> > > configurable
> > > > > via
> > > > > > > > > > > > > configuration?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900.e Is the leader epoch cache offloaded with
> > > > every
> > > > > > > > segment?
> > > > > > > > > > Or
> > > > > > > > > > > > when
> > > > > > > > > > > > > > > > a new checkpoint is detected? If that
> > information
> > > > is
> > > > > not
> > > > > > > > > always
> > > > > > > > > > > > > > > > offloaded to avoid duplicating data, how does
> > the
> > > > > remote
> > > > > > > > > > storage
> > > > > > > > > > > > > > > > satisfy the request to retrieve it?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900.f Since the leader epoch cache covers the
> > > > entire
> > > > > > > > replica
> > > > > > > > > > > > lineage,
> > > > > > > > > > > > > > > > what happens if, after a leader epoch cache
> > file
> > > is
> > > > > > > > offloaded
> > > > > > > > > > with
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > given segment, the local epoch cache is
> > truncated
> > > > > [not
> > > > > > > > > > necessarily
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > a range of offset included in tiered segments]?
> > > How
> > > > > are
> > > > > > > > > remote
> > > > > > > > > > and
> > > > > > > > > > > > > > > > local leader epoch caches kept consistent?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900.g Consumer can also use leader epochs (e.g.
> > > to
> > > > > enable
> > > > > > > > > > fencing
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > protect against stale leaders). What
> > differences
> > > > > would
> > > > > > > > there
> > > > > > > > > be
> > > > > > > > > > > > > > > > between consumer and follower fetches?
> > > Especially,
> > > > > would
> > > > > > > > > > consumers
> > > > > > > > > > > > > > > > also fetch leader epoch information from the
> > > remote
> > > > > > > > storage?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 900.h Assume a newly elected leader of a
> > > > > topic-partition
> > > > > > > > > > detects
> > > > > > > > > > > > more
> > > > > > > > > > > > > > > > recent segments are available in the external
> > > > > storage,
> > > > > > > with
> > > > > > > > > > epochs
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > > its local epoch. Does it ignore these segments
> > > and
> > > > > their
> > > > > > > > > > associated
> > > > > > > > > > > > > > > > epoch-to-offset vectors? Or try to reconstruct
> > > its
> > > > > local
> > > > > > > > > > replica
> > > > > > > > > > > > > > > > lineage based on the data remotely available?
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Alexandre
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > [1]
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> > https://docs.google.com/document/d/18tnobSas3mKFZFr8oRguZoj_tkD_sGzivuLRlMloEMs/edit?usp=sharing
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Le jeu. 4 juin 2020 à 19:55, Satish Duggana <
> > > > > > > > > > > > > satish.duggana@gmail.com>
> > > > > > > > > > > > > > > a écrit :
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi Jun,
> > > > > > > > > > > > > > > > > Please let us know if you have any comments
> > on
> > > > > > > > > "transactional
> > > > > > > > > > > > > > support"
> > > > > > > > > > > > > > > > > and "follower requests/replication" mentioned
> > > in
> > > > > the
> > > > > > > > wiki.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Jun 2, 2020 at 9:25 PM Satish
> > Duggana <
> > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks Jun for your comments.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >100. It would be useful to provide more
> > > > details
> > > > > on
> > > > > > > how
> > > > > > > > > > those
> > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to really
> > > > assess
> > > > > > > > whether
> > > > > > > > > > the
> > > > > > > > > > > > new
> > > > > > > > > > > > > > > apis are sufficient/redundant. A few examples
> > > below.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > We will update the wiki and let you know.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >100.1 deleteRecords seems to only advance
> > > the
> > > > > > > > > > logStartOffset
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > Log. How does that trigger the deletion of remote
> > > log
> > > > > > > > segments?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > RLMTask for leader partition periodically
> > > > checks
> > > > > > > > whether
> > > > > > > > > > there
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > remote log segments earlier to
> > logStartOffset
> > > > > and the
> > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > remote log segment metadata and data are
> > > > deleted
> > > > > by
> > > > > > > > using
> > > > > > > > > > RLMM
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > RSM.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >100.2 stopReplica with deletion is used
> > in 2
> > > > > cases
> > > > > > > (a)
> > > > > > > > > > replica
> > > > > > > > > > > > > > > reassignment; (b) topic deletion. We only want to
> > > > > delete
> > > > > > > the
> > > > > > > > > > tiered
> > > > > > > > > > > > > > > metadata in the second case. Also, in the second
> > > > case,
> > > > > who
> > > > > > > > > > initiates
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > deletion of the remote segment since the leader
> > may
> > > > not
> > > > > > > > exist?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Right, it is deleted only incase of topic
> > > > > deletion
> > > > > > > > only.
> > > > > > > > > We
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > cover
> > > > > > > > > > > > > > > > > > the details in the KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >100.3 "LogStartOffset of a topic can be
> > > either
> > > > > in
> > > > > > > > local
> > > > > > > > > > or in
> > > > > > > > > > > > > > > remote storage." If LogStartOffset exists in both
> > > > > places,
> > > > > > > > which
> > > > > > > > > > one
> > > > > > > > > > > > is
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > source of truth?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I meant the logStartOffset can point to
> > > either
> > > > of
> > > > > > > local
> > > > > > > > > > segment
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > remote segment but it is initialised and
> > > > > maintained
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > Log
> > > > > > > > > > > > > > class
> > > > > > > > > > > > > > > > > > like now.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >100.4 List<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > topicPartition,
> > > > > long
> > > > > > > > > > minOffset):
> > > > > > > > > > > > > How
> > > > > > > > > > > > > > > is minOffset supposed to be used?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Returns list of remote segments, sorted by
> > > > > baseOffset
> > > > > > > > in
> > > > > > > > > > > > > ascending
> > > > > > > > > > > > > > > > > > order that have baseOffset >= the given min
> > > > > Offset.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >100.5 When copying a segment to remote
> > > > storage,
> > > > > it
> > > > > > > > seems
> > > > > > > > > > we
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > calling the same RLMM.putRemoteLogSegmentData()
> > > twice
> > > > > > > before
> > > > > > > > > and
> > > > > > > > > > > > after
> > > > > > > > > > > > > > > copyLogSegment(). Could you explain why?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > This is more about prepare/commit/rollback
> > as
> > > > you
> > > > > > > > > > suggested.
> > > > > > > > > > > > We
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > update the wiki with the new APIs.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >100.6 LogSegmentData includes
> > > > leaderEpochCache,
> > > > > but
> > > > > > > > > there
> > > > > > > > > > is
> > > > > > > > > > > > no
> > > > > > > > > > > > > > api
> > > > > > > > > > > > > > > in RemoteStorageManager to retrieve it.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Nice catch, copy/paste issue. There is an
> > API
> > > > to
> > > > > > > > retrieve
> > > > > > > > > > it.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >101. If the __remote_log_metadata is for
> > > > > production
> > > > > > > > > usage,
> > > > > > > > > > > > could
> > > > > > > > > > > > > > > you provide more details? For example, what is
> > the
> > > > > schema
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > data
> > > > > > > > > > > > > > (both
> > > > > > > > > > > > > > > key and value)? How is the topic
> > maintained,delete
> > > or
> > > > > > > > compact?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > It is with delete config and it’s retention
> > > > > period is
> > > > > > > > > > suggested
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > more than the remote retention period.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >110. Is the cache implementation in
> > > > > > > > > > RemoteLogMetadataManager
> > > > > > > > > > > > > meant
> > > > > > > > > > > > > > > for production usage? If so, could you provide
> > more
> > > > > details
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > and how/where the data is stored?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > The proposal is to have a cache (with
> > default
> > > > > > > > > > implementation
> > > > > > > > > > > > > backed
> > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > rocksdb) but it will be added in later
> > > > versions.
> > > > > We
> > > > > > > > will
> > > > > > > > > > add
> > > > > > > > > > > > this
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > future work items.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >111. "Committed offsets can be stored in a
> > > > local
> > > > > > > > file".
> > > > > > > > > > Could
> > > > > > > > > > > > > you
> > > > > > > > > > > > > > > describe the format of the file and where it's
> > > > stored?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > We will cover this in the KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >112. Truncation of remote segments under
> > > > unclean
> > > > > > > > leader
> > > > > > > > > > > > > election:
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > am not sure who figures out the truncated remote
> > > > > segments
> > > > > > > and
> > > > > > > > > how
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > information is propagated to all replicas?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > We will add this in detail in the KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >113. "If there are any failures in
> > removing
> > > > > remote
> > > > > > > log
> > > > > > > > > > > > segments
> > > > > > > > > > > > > > > then those are stored in a specific topic
> > (default
> > > as
> > > > > > > > > > > > > > > __remote_segments_to_be_deleted)". Is it
> > necessary
> > > to
> > > > > add
> > > > > > > yet
> > > > > > > > > > another
> > > > > > > > > > > > > > > internal topic? Could we just keep retrying?
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > This is not really an internal topic, it
> > will
> > > > be
> > > > > > > > exposed
> > > > > > > > > > as a
> > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > configurable topic. After a few retries, we
> > > > want
> > > > > user
> > > > > > > > to
> > > > > > > > > > know
> > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > > > the failure so that they can take an action
> > > > > later by
> > > > > > > > > > consuming
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > this topic. We want to keep this simple
> > > instead
> > > > > of
> > > > > > > > > retrying
> > > > > > > > > > > > > > > > > > continuously and maintaining the deletion
> > > state
> > > > > etc.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >114. "We may not need to copy
> > > > > producer-id-snapshot
> > > > > > > as
> > > > > > > > we
> > > > > > > > > > are
> > > > > > > > > > > > > > > copying only segments earlier to
> > > last-stable-offset."
> > > > > Hmm,
> > > > > > > > not
> > > > > > > > > > sure
> > > > > > > > > > > > > about
> > > > > > > > > > > > > > > that. The producer snapshot includes things like
> > > the
> > > > > last
> > > > > > > > > > timestamp
> > > > > > > > > > > > of
> > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > open producer id and can affect when those
> > producer
> > > > > ids are
> > > > > > > > > > expired.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Sure, this will be added as part of the
> > > > > > > LogSegmentData.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Satish.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Fri, May 29, 2020 at 6:39 AM Jun Rao <
> > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Hi, Satish,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Made another pass on the wiki. A few more
> > > > > comments
> > > > > > > > > below.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 100. It would be useful to provide more
> > > > > details on
> > > > > > > > how
> > > > > > > > > > those
> > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > are used. Otherwise, it's kind of hard to really
> > > > assess
> > > > > > > > whether
> > > > > > > > > > the
> > > > > > > > > > > > new
> > > > > > > > > > > > > > > apis are sufficient/redundant. A few examples
> > > below.
> > > > > > > > > > > > > > > > > > > 100.1 deleteRecords seems to only advance
> > > the
> > > > > > > > > > logStartOffset
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > Log. How does that trigger the deletion of remote
> > > log
> > > > > > > > segments?
> > > > > > > > > > > > > > > > > > > 100.2 stopReplica with deletion is used
> > in
> > > 2
> > > > > cases
> > > > > > > > (a)
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > > reassignment; (b) topic deletion. We only want to
> > > > > delete
> > > > > > > the
> > > > > > > > > > tiered
> > > > > > > > > > > > > > > metadata in the second case. Also, in the second
> > > > case,
> > > > > who
> > > > > > > > > > initiates
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > deletion of the remote segment since the leader
> > may
> > > > not
> > > > > > > > exist?
> > > > > > > > > > > > > > > > > > > 100.3 "LogStartOffset of a topic can be
> > > > either
> > > > > in
> > > > > > > > local
> > > > > > > > > > or in
> > > > > > > > > > > > > > > remote storage." If LogStartOffset exists in both
> > > > > places,
> > > > > > > > which
> > > > > > > > > > one
> > > > > > > > > > > > is
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > source of truth?
> > > > > > > > > > > > > > > > > > > 100.4 List<RemoteLogSegmentMetadata>
> > > > > > > > > > > > > > > listRemoteLogSegments(TopicPartition
> > > topicPartition,
> > > > > long
> > > > > > > > > > minOffset):
> > > > > > > > > > > > > How
> > > > > > > > > > > > > > > is minOffset supposed to be used?
> > > > > > > > > > > > > > > > > > > 100.5 When copying a segment to remote
> > > > > storage, it
> > > > > > > > > seems
> > > > > > > > > > we
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > calling the same RLMM.putRemoteLogSegmentData()
> > > twice
> > > > > > > before
> > > > > > > > > and
> > > > > > > > > > > > after
> > > > > > > > > > > > > > > copyLogSegment(). Could you explain why?
> > > > > > > > > > > > > > > > > > > 100.6 LogSegmentData includes
> > > > > leaderEpochCache, but
> > > > > > > > > > there is
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > api in RemoteStorageManager to retrieve it.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 101. If the __remote_log_metadata is for
> > > > > production
> > > > > > > > > > usage,
> > > > > > > > > > > > > could
> > > > > > > > > > > > > > > you provide more details? For example, what is
> > the
> > > > > schema
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > data
> > > > > > > > > > > > > > (both
> > > > > > > > > > > > > > > key and value)? How is the topic
> > maintained,delete
> > > or
> > > > > > > > compact?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 110. Is the cache implementation in
> > > > > > > > > > RemoteLogMetadataManager
> > > > > > > > > > > > > > meant
> > > > > > > > > > > > > > > for production usage? If so, could you provide
> > more
> > > > > details
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > and how/where the data is stored?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 111. "Committed offsets can be stored in
> > a
> > > > > local
> > > > > > > > file".
> > > > > > > > > > Could
> > > > > > > > > > > > > you
> > > > > > > > > > > > > > > describe the format of the file and where it's
> > > > stored?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 112. Truncation of remote segments under
> > > > > unclean
> > > > > > > > leader
> > > > > > > > > > > > > election:
> > > > > > > > > > > > > > > I am not sure who figures out the truncated
> > remote
> > > > > segments
> > > > > > > > and
> > > > > > > > > > how
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > information is propagated to all replicas?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 113. "If there are any failures in
> > removing
> > > > > remote
> > > > > > > > log
> > > > > > > > > > > > segments
> > > > > > > > > > > > > > > then those are stored in a specific topic
> > (default
> > > as
> > > > > > > > > > > > > > > __remote_segments_to_be_deleted)". Is it
> > necessary
> > > to
> > > > > add
> > > > > > > yet
> > > > > > > > > > another
> > > > > > > > > > > > > > > internal topic? Could we just keep retrying?
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 114. "We may not need to copy
> > > > > producer-id-snapshot
> > > > > > > as
> > > > > > > > > we
> > > > > > > > > > are
> > > > > > > > > > > > > > > copying only segments earlier to
> > > last-stable-offset."
> > > > > Hmm,
> > > > > > > > not
> > > > > > > > > > sure
> > > > > > > > > > > > > about
> > > > > > > > > > > > > > > that. The producer snapshot includes things like
> > > the
> > > > > last
> > > > > > > > > > timestamp
> > > > > > > > > > > > of
> > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > open producer id and can affect when those
> > producer
> > > > > ids are
> > > > > > > > > > expired.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Thu, May 28, 2020 at 5:38 AM Satish
> > > > Duggana
> > > > > <
> > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> Hi Jun,
> > > > > > > > > > > > > > > > > > >> Gentle reminder. Please go through the
> > > > updated
> > > > > > > wiki
> > > > > > > > > and
> > > > > > > > > > let
> > > > > > > > > > > > us
> > > > > > > > > > > > > > > know your comments.
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> Thanks,
> > > > > > > > > > > > > > > > > > >> Satish.
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> On Tue, May 19, 2020 at 3:50 PM Satish
> > > > > Duggana <
> > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > >>> Hi Jun,
> > > > > > > > > > > > > > > > > > >>> Please go through the wiki which has
> > the
> > > > > latest
> > > > > > > > > > updates.
> > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > doc is updated frequently to be in sync with
> > wiki.
> > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > >>> Thanks,
> > > > > > > > > > > > > > > > > > >>> Satish.
> > > > > > > > > > > > > > > > > > >>>
> > > > > > > > > > > > > > > > > > >>> On Tue, May 19, 2020 at 12:30 AM Jun
> > Rao
> > > <
> > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > >>>> Hi, Satish,
> > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > >>>> Thanks for the update. Just to
> > clarify.
> > > > > Which
> > > > > > > doc
> > > > > > > > > has
> > > > > > > > > > the
> > > > > > > > > > > > > > > latest updates, the wiki or the google doc?
> > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > >>>> Jun
> > > > > > > > > > > > > > > > > > >>>>
> > > > > > > > > > > > > > > > > > >>>> On Thu, May 14, 2020 at 10:38 AM
> > Satish
> > > > > Duggana
> > > > > > > <
> > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > >>>>> Thanks for your comments.  We updated
> > > the
> > > > > KIP
> > > > > > > > with
> > > > > > > > > > more
> > > > > > > > > > > > > > > details.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> >100. For each of the operations
> > > related
> > > > to
> > > > > > > > > tiering,
> > > > > > > > > > it
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be useful to provide a description on how it
> > works
> > > > > with the
> > > > > > > > new
> > > > > > > > > > API.
> > > > > > > > > > > > > > These
> > > > > > > > > > > > > > > include things like consumer fetch, replica
> > fetch,
> > > > > > > > > > > > offsetForTimestamp,
> > > > > > > > > > > > > > > retention (remote and local) by size, time and
> > > > > > > > logStartOffset,
> > > > > > > > > > topic
> > > > > > > > > > > > > > > deletion, etc. This will tell us if the proposed
> > > APIs
> > > > > are
> > > > > > > > > > sufficient.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> We addressed most of these APIs in
> > the
> > > > > KIP. We
> > > > > > > > can
> > > > > > > > > > add
> > > > > > > > > > > > more
> > > > > > > > > > > > > > > details if needed.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> >101. For the default implementation
> > > > based
> > > > > on
> > > > > > > > > > internal
> > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > is it meant as a proof of concept or for
> > production
> > > > > usage?
> > > > > > > I
> > > > > > > > > > assume
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > it's the former. However, if it's the latter,
> > then
> > > > the
> > > > > KIP
> > > > > > > > > needs
> > > > > > > > > > to
> > > > > > > > > > > > > > > describe the design in more detail.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> It is production usage as was
> > mentioned
> > > > in
> > > > > an
> > > > > > > > > earlier
> > > > > > > > > > > > mail.
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > plan to update this section in the next few days.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> >102. When tiering a segment, the
> > > segment
> > > > > is
> > > > > > > > first
> > > > > > > > > > > > written
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the object store and then its metadata is written
> > > to
> > > > > RLMM
> > > > > > > > using
> > > > > > > > > > the
> > > > > > > > > > > > api
> > > > > > > > > > > > > > > "void putRemoteLogSegmentData()". One potential
> > > issue
> > > > > with
> > > > > > > > this
> > > > > > > > > > > > > approach
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > that if the system fails after the first
> > operation,
> > > > it
> > > > > > > > leaves a
> > > > > > > > > > > > garbage
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the object store that's never reclaimed. One way
> > to
> > > > > improve
> > > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > > to
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > two separate APIs, sth like
> > > > > > > preparePutRemoteLogSegmentData()
> > > > > > > > > and
> > > > > > > > > > > > > > > commitPutRemoteLogSegmentData().
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> That is a good point. We currently
> > > have a
> > > > > > > > different
> > > > > > > > > > way
> > > > > > > > > > > > > using
> > > > > > > > > > > > > > > markers in the segment but your suggestion is
> > much
> > > > > better.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> >103. It seems that the transactional
> > > > > support
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > > > ability
> > > > > > > > > > > > > > > to read from follower are missing.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> KIP is updated with transactional
> > > > support,
> > > > > > > > follower
> > > > > > > > > > fetch
> > > > > > > > > > > > > > > semantics, and reading from a follower.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> >104. It would be useful to provide a
> > > > > testing
> > > > > > > > plan
> > > > > > > > > > for
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> We added a few tests by introducing
> > > test
> > > > > util
> > > > > > > for
> > > > > > > > > > tiered
> > > > > > > > > > > > > > > storage in the PR. We will provide the testing
> > plan
> > > > in
> > > > > the
> > > > > > > > next
> > > > > > > > > > few
> > > > > > > > > > > > > days.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> Thanks,
> > > > > > > > > > > > > > > > > > >>>>> Satish.
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>>
> > > > > > > > > > > > > > > > > > >>>>> On Wed, Feb 26, 2020 at 9:43 PM
> > Harsha
> > > > > > > > > Chintalapani <
> > > > > > > > > > > > > > > kafka@harsha.io> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>> On Tue, Feb 25, 2020 at 12:46 PM,
> > Jun
> > > > Rao
> > > > > <
> > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the updated doc. The new
> > > API
> > > > > seems
> > > > > > > > to
> > > > > > > > > > be an
> > > > > > > > > > > > > > > improvement overall. A few more comments below.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 100. For each of the operations
> > > related
> > > > > to
> > > > > > > > > > tiering, it
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be useful to provide a description on how it
> > works
> > > > > with the
> > > > > > > > new
> > > > > > > > > > API.
> > > > > > > > > > > > > > These
> > > > > > > > > > > > > > > include things like consumer fetch, replica
> > fetch,
> > > > > > > > > > > > offsetForTimestamp,
> > > > > > > > > > > > > > > retention
> > > > > > > > > > > > > > > > > > >>>>>>> (remote and local) by size, time
> > and
> > > > > > > > > > logStartOffset,
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > deletion, etc. This will tell us if the proposed
> > > APIs
> > > > > are
> > > > > > > > > > sufficient.
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>> Thanks for the feedback Jun. We will
> > > add
> > > > > more
> > > > > > > > > > details
> > > > > > > > > > > > > around
> > > > > > > > > > > > > > > this.
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 101. For the default implementation
> > > > > based on
> > > > > > > > > > internal
> > > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > is it meant as a proof of concept or for
> > production
> > > > > usage?
> > > > > > > I
> > > > > > > > > > assume
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > it's the former. However, if it's the latter,
> > then
> > > > the
> > > > > KIP
> > > > > > > > > needs
> > > > > > > > > > to
> > > > > > > > > > > > > > > describe the design in more detail.
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>> Yes it meant to be for production
> > use.
> > > > > > > Ideally
> > > > > > > > it
> > > > > > > > > > would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > good to merge this in as the default
> > implementation
> > > > for
> > > > > > > > > metadata
> > > > > > > > > > > > > service.
> > > > > > > > > > > > > > > We can add more details around design and
> > testing.
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 102. When tiering a segment, the
> > > > segment
> > > > > is
> > > > > > > > first
> > > > > > > > > > > > written
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the object store and then its metadata is written
> > > to
> > > > > RLMM
> > > > > > > > using
> > > > > > > > > > the
> > > > > > > > > > > > api
> > > > > > > > > > > > > > > "void putRemoteLogSegmentData()".
> > > > > > > > > > > > > > > > > > >>>>>>> One potential issue with this
> > > approach
> > > > is
> > > > > > > that
> > > > > > > > if
> > > > > > > > > > the
> > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > fails after the first operation, it leaves a
> > > garbage
> > > > > in the
> > > > > > > > > > object
> > > > > > > > > > > > > store
> > > > > > > > > > > > > > > that's never reclaimed. One way to improve this
> > is
> > > to
> > > > > have
> > > > > > > > two
> > > > > > > > > > > > separate
> > > > > > > > > > > > > > > APIs, sth like preparePutRemoteLogSegmentData()
> > and
> > > > > > > > > > > > > > > commitPutRemoteLogSegmentData().
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 103. It seems that the
> > transactional
> > > > > support
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > > > > ability
> > > > > > > > > > > > > > > to read from follower are missing.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 104. It would be useful to provide
> > a
> > > > > testing
> > > > > > > > plan
> > > > > > > > > > for
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>> We are working on adding more
> > details
> > > > > around
> > > > > > > > > > > > transactional
> > > > > > > > > > > > > > > support and coming up with test plan.
> > > > > > > > > > > > > > > > > > >>>>>> Add system tests and integration
> > > tests.
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Mon, Feb 24, 2020 at 8:10 AM
> > > Satish
> > > > > > > Duggana
> > > > > > > > <
> > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > >>>>>>> Please look at the earlier reply
> > and
> > > > let
> > > > > us
> > > > > > > > know
> > > > > > > > > > your
> > > > > > > > > > > > > > > comments.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > >>>>>>> Satish.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Wed, Feb 12, 2020 at 4:06 PM
> > > Satish
> > > > > > > Duggana
> > > > > > > > <
> > > > > > > > > > > > > > > satish.duggana@gmail.com> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks for your comments on the
> > > > > separation of
> > > > > > > > > > remote
> > > > > > > > > > > > log
> > > > > > > > > > > > > > > metadata storage and remote log storage.
> > > > > > > > > > > > > > > > > > >>>>>>> We had a few discussions since
> > early
> > > > Jan
> > > > > on
> > > > > > > how
> > > > > > > > > to
> > > > > > > > > > > > > support
> > > > > > > > > > > > > > > eventually consistent stores like S3 by
> > uncoupling
> > > > > remote
> > > > > > > log
> > > > > > > > > > segment
> > > > > > > > > > > > > > > metadata and remote log storage. It is written
> > with
> > > > > details
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > doc
> > > > > > > > > > > > > > > here(1). Below is the brief summary of the
> > > discussion
> > > > > from
> > > > > > > > that
> > > > > > > > > > doc.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> The current approach consists of
> > > > pulling
> > > > > the
> > > > > > > > > > remote log
> > > > > > > > > > > > > > > segment metadata from remote log storage APIs. It
> > > > > worked
> > > > > > > fine
> > > > > > > > > for
> > > > > > > > > > > > > > storages
> > > > > > > > > > > > > > > like HDFS. But one of the problems of relying on
> > > the
> > > > > remote
> > > > > > > > > > storage
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > maintain metadata is that tiered-storage needs to
> > > be
> > > > > > > strongly
> > > > > > > > > > > > > consistent,
> > > > > > > > > > > > > > > with an impact not only on the metadata(e.g. LIST
> > > in
> > > > > S3)
> > > > > > > but
> > > > > > > > > > also on
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > segment data(e.g. GET after a DELETE in S3). The
> > > cost
> > > > > of
> > > > > > > > > > maintaining
> > > > > > > > > > > > > > > metadata in remote storage needs to be factored
> > in.
> > > > > This is
> > > > > > > > > true
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > case of S3, LIST APIs incur huge costs as you
> > > raised
> > > > > > > earlier.
> > > > > > > > > > > > > > > > > > >>>>>>> So, it is good to separate the
> > remote
> > > > > storage
> > > > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > > > > > > remote log metadata store. We refactored the
> > > existing
> > > > > > > > > > > > > > RemoteStorageManager
> > > > > > > > > > > > > > > and introduced RemoteLogMetadataManager. Remote
> > log
> > > > > > > metadata
> > > > > > > > > > store
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > give strong consistency semantics but remote log
> > > > > storage
> > > > > > > can
> > > > > > > > be
> > > > > > > > > > > > > > eventually
> > > > > > > > > > > > > > > consistent.
> > > > > > > > > > > > > > > > > > >>>>>>> We can have a default
> > implementation
> > > > for
> > > > > > > > > > > > > > > RemoteLogMetadataManager which uses an internal
> > > > > topic(as
> > > > > > > > > > mentioned in
> > > > > > > > > > > > > one
> > > > > > > > > > > > > > > of our earlier emails) as storage. But users can
> > > > always
> > > > > > > > plugin
> > > > > > > > > > their
> > > > > > > > > > > > > own
> > > > > > > > > > > > > > > RemoteLogMetadataManager implementation based on
> > > > their
> > > > > > > > > > environment.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Please go through the updated KIP
> > and
> > > > > let us
> > > > > > > > know
> > > > > > > > > > your
> > > > > > > > > > > > > > > comments. We have started refactoring for the
> > > changes
> > > > > > > > mentioned
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > and there may be a few more updates to the APIs.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> [1]
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> > https://docs.google.com/document/d/1qfkBCWL1e7ZWkHU7brxKDBebq4ie9yK20XJnKbgAlew/edit?ts=5e208ec7#
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Dec 27, 2019 at 5:43 PM
> > Ivan
> > > > > > > Yurchenko
> > > > > > > > <
> > > > > > > > > > > > > > > ivan0yurchenko@gmail.com>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi all,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Jun:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (a) Cost: S3 list object requests
> > > cost
> > > > > $0.005
> > > > > > > > per
> > > > > > > > > > 1000
> > > > > > > > > > > > > > > requests. If
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> have 100,000 partitions and want to
> > > > pull
> > > > > the
> > > > > > > > > > metadata
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> partition
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> at
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the rate of 1/sec. It can cost
> > > > $0.5/sec,
> > > > > > > which
> > > > > > > > is
> > > > > > > > > > > > roughly
> > > > > > > > > > > > > > > $40K per
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> day.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> I want to note here, that no
> > > reasonably
> > > > > > > durable
> > > > > > > > > > storage
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > be cheap at 100k RPS. For example, DynamoDB might
> > > > give
> > > > > the
> > > > > > > > same
> > > > > > > > > > > > > ballpark
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> figures.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> If we want to keep the pull-based
> > > > > approach,
> > > > > > > we
> > > > > > > > > can
> > > > > > > > > > try
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > reduce this
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> number
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> in several ways: doing listings
> > less
> > > > > > > frequently
> > > > > > > > > (as
> > > > > > > > > > > > > Satish
> > > > > > > > > > > > > > > mentioned, with the current defaults it's ~3.33k
> > > RPS
> > > > > for
> > > > > > > your
> > > > > > > > > > > > example),
> > > > > > > > > > > > > > > batching listing operations in some way
> > (depending
> > > on
> > > > > the
> > > > > > > > > > storage; it
> > > > > > > > > > > > > > might
> > > > > > > > > > > > > > > require the change of RSM's interface).
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> There are different ways for doing
> > > push
> > > > > based
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > propagation.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Some
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> object stores may support that
> > > already.
> > > > > For
> > > > > > > > > > example, S3
> > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> events
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> notification
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> This sounds interesting. However, I
> > > > see a
> > > > > > > > couple
> > > > > > > > > of
> > > > > > > > > > > > > issues
> > > > > > > > > > > > > > > using it:
> > > > > > > > > > > > > > > > > > >>>>>>> 1. As I understand the
> > documentation,
> > > > > > > > > notification
> > > > > > > > > > > > > delivery
> > > > > > > > > > > > > > > is not guaranteed
> > > > > > > > > > > > > > > > > > >>>>>>> and it's recommended to
> > periodically
> > > do
> > > > > LIST
> > > > > > > to
> > > > > > > > > > fill
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > gaps. Which brings us back to the same LIST
> > > > consistency
> > > > > > > > > > guarantees
> > > > > > > > > > > > > issue.
> > > > > > > > > > > > > > > > > > >>>>>>> 2. The same goes for the broker
> > > start:
> > > > > to get
> > > > > > > > the
> > > > > > > > > > > > current
> > > > > > > > > > > > > > > state, we
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> need
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to LIST.
> > > > > > > > > > > > > > > > > > >>>>>>> 3. The dynamic set of multiple
> > > > consumers
> > > > > > > > (RSMs):
> > > > > > > > > > AFAIK
> > > > > > > > > > > > > SQS
> > > > > > > > > > > > > > > and SNS
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> aren't
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> designed for such a case.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Alexandre:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> A.1 As commented on PR 7561, S3
> > > > > consistency
> > > > > > > > model
> > > > > > > > > > > > [1][2]
> > > > > > > > > > > > > > > implies RSM
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> cannot
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> relies solely on S3 APIs to
> > guarantee
> > > > the
> > > > > > > > > expected
> > > > > > > > > > > > strong
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> consistency. The
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> proposed implementation [3] would
> > > need
> > > > > to be
> > > > > > > > > > updated to
> > > > > > > > > > > > > > take
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> into
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> account. Let’s talk more about
> > this.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thank you for the feedback. I
> > clearly
> > > > > see the
> > > > > > > > > need
> > > > > > > > > > for
> > > > > > > > > > > > > > > changing the S3 implementation
> > > > > > > > > > > > > > > > > > >>>>>>> to provide stronger consistency
> > > > > guarantees.
> > > > > > > As
> > > > > > > > it
> > > > > > > > > > see
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > this thread, there are
> > > > > > > > > > > > > > > > > > >>>>>>> several possible approaches to
> > this.
> > > > > Let's
> > > > > > > > > discuss
> > > > > > > > > > > > > > > RemoteLogManager's contract and
> > > > > > > > > > > > > > > > > > >>>>>>> behavior (like pull vs push model)
> > > > > further
> > > > > > > > before
> > > > > > > > > > > > picking
> > > > > > > > > > > > > > > one (or
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> several -
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> ?) of them.
> > > > > > > > > > > > > > > > > > >>>>>>> I'm going to do some evaluation of
> > > > > DynamoDB
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > > > > > > pull-based
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> approach,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> if it's possible to apply it
> > paying a
> > > > > > > > reasonable
> > > > > > > > > > bill.
> > > > > > > > > > > > > > Also,
> > > > > > > > > > > > > > > of the push-based approach
> > > > > > > > > > > > > > > > > > >>>>>>> with a Kafka topic as the medium.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> A.2.3 Atomicity – what does an
> > > > > implementation
> > > > > > > > of
> > > > > > > > > > RSM
> > > > > > > > > > > > need
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> respect to atomicity of the APIs
> > > > > > > > copyLogSegment,
> > > > > > > > > > > > > > > cleanupLogUntil and deleteTopicPartition? If a
> > > > partial
> > > > > > > > failure
> > > > > > > > > > > > happens
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > any of those
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (e.g.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the S3 implementation, if one of
> > the
> > > > > multiple
> > > > > > > > > > uploads
> > > > > > > > > > > > > fails
> > > > > > > > > > > > > > > [4]),
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> The S3 implementation is going to
> > > > > change, but
> > > > > > > > > it's
> > > > > > > > > > > > worth
> > > > > > > > > > > > > > > clarifying
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> anyway.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> The segment log file is being
> > > uploaded
> > > > > after
> > > > > > > S3
> > > > > > > > > has
> > > > > > > > > > > > acked
> > > > > > > > > > > > > > > uploading of all other files associated with the
> > > > > segment
> > > > > > > and
> > > > > > > > > only
> > > > > > > > > > > > after
> > > > > > > > > > > > > > > this the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> whole
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> segment file set becomes visible
> > > > > remotely for
> > > > > > > > > > > > operations
> > > > > > > > > > > > > > > like listRemoteSegments [1].
> > > > > > > > > > > > > > > > > > >>>>>>> In case of upload failure, the
> > files
> > > > > that has
> > > > > > > > > been
> > > > > > > > > > > > > > > successfully
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> uploaded
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> stays
> > > > > > > > > > > > > > > > > > >>>>>>> as invisible garbage that is
> > > collected
> > > > by
> > > > > > > > > > > > cleanupLogUntil
> > > > > > > > > > > > > > (or
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> overwritten
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> successfully later).
> > > > > > > > > > > > > > > > > > >>>>>>> And the opposite happens during the
> > > > > deletion:
> > > > > > > > log
> > > > > > > > > > files
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > deleted
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> first.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> This approach should generally work
> > > > when
> > > > > we
> > > > > > > > solve
> > > > > > > > > > > > > > > consistency issues by adding a strongly
> > consistent
> > > > > > > storage: a
> > > > > > > > > > > > segment's
> > > > > > > > > > > > > > > uploaded files
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> remain
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> invisible garbage until some
> > metadata
> > > > > about
> > > > > > > > them
> > > > > > > > > is
> > > > > > > > > > > > > > written.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> A.3 Caching – storing locally the
> > > > > segments
> > > > > > > > > > retrieved
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > the remote storage is excluded as it does not
> > align
> > > > > with
> > > > > > > the
> > > > > > > > > > original
> > > > > > > > > > > > > > intent
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> and even
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> defeat some of its purposes (save
> > > disk
> > > > > space
> > > > > > > > > etc.).
> > > > > > > > > > > > That
> > > > > > > > > > > > > > > said, could
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> there
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> be other types of use cases where
> > the
> > > > > pattern
> > > > > > > > of
> > > > > > > > > > access
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> remotely
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> stored segments would benefit from
> > > > local
> > > > > > > > caching
> > > > > > > > > > (and
> > > > > > > > > > > > > > > potentially read-ahead)? Consider the use case
> > of a
> > > > > large
> > > > > > > > pool
> > > > > > > > > of
> > > > > > > > > > > > > > consumers
> > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> start
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> a backfill at the same time for one
> > > day
> > > > > worth
> > > > > > > > of
> > > > > > > > > > data
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > one year
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> ago
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> stored remotely. Caching the
> > segments
> > > > > locally
> > > > > > > > > would
> > > > > > > > > > > > allow
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> uncouple the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> load on the remote storage from the
> > > > load
> > > > > on
> > > > > > > the
> > > > > > > > > > Kafka
> > > > > > > > > > > > > > > cluster. Maybe
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> RLM could expose a configuration
> > > > > parameter to
> > > > > > > > > > switch
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > feature
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> on/off?
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> I tend to agree here, caching
> > remote
> > > > > segments
> > > > > > > > > > locally
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > making this configurable sounds pretty practical
> > to
> > > > > me. We
> > > > > > > > > should
> > > > > > > > > > > > > > implement
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> this,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> maybe not in the first iteration.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Br,
> > > > > > > > > > > > > > > > > > >>>>>>> Ivan
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> [1]
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> > https://github.com/harshach/kafka/pull/18/files#diff-4d73d01c16caed6f2548fc3063550ef0R152
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Thu, 19 Dec 2019 at 19:49,
> > > Alexandre
> > > > > > > > Dupriez <
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> alexandre.dupriez@gmail.com>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thank you for the feedback. I am
> > > trying
> > > > > to
> > > > > > > > > > understand
> > > > > > > > > > > > > how a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> push-based
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> approach would work.
> > > > > > > > > > > > > > > > > > >>>>>>> In order for the metadata to be
> > > > > propagated
> > > > > > > > (under
> > > > > > > > > > the
> > > > > > > > > > > > > > > assumption you stated), would you plan to add a
> > new
> > > > > API in
> > > > > > > > > Kafka
> > > > > > > > > > to
> > > > > > > > > > > > > allow
> > > > > > > > > > > > > > > the metadata store to send them directly to the
> > > > > brokers?
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > >>>>>>> Alexandre
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Le mer. 18 déc. 2019 à 20:14, Jun
> > > Rao <
> > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > écrit :
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Ivan,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> There are different ways for doing
> > > push
> > > > > based
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> propagation. Some
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> object stores may support that
> > > already.
> > > > > For
> > > > > > > > > > example, S3
> > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> events
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> notification (
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > >
> > > > >
> > https://docs.aws.amazon.com/AmazonS3/latest/dev/NotificationHowTo.html
> > > > > > > > > > > > > ).
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Otherwise one could use a separate
> > > > > metadata
> > > > > > > > store
> > > > > > > > > > that
> > > > > > > > > > > > > > > supports
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> push-based
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> change propagation. Other people
> > have
> > > > > > > mentioned
> > > > > > > > > > using a
> > > > > > > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> topic. The
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> best approach may depend on the
> > > object
> > > > > store
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > > > > > operational environment (e.g. whether an external
> > > > > metadata
> > > > > > > > > store
> > > > > > > > > > is
> > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> available).
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> The above discussion is based on
> > the
> > > > > > > assumption
> > > > > > > > > > that we
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> cache the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> object metadata locally in every
> > > > broker.
> > > > > I
> > > > > > > > > > mentioned
> > > > > > > > > > > > > > earlier
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> an
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> alternative is to just
> > store/retrieve
> > > > > those
> > > > > > > > > > metadata in
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > external metadata store. That may simplify the
> > > > > > > implementation
> > > > > > > > > in
> > > > > > > > > > some
> > > > > > > > > > > > > > cases.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Dec 5, 2019 at 7:01 AM
> > Satish
> > > > > > > Duggana <
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> satish.duggana@gmail.com>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks for your reply.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Currently, `listRemoteSegments` is
> > > > > called at
> > > > > > > > the
> > > > > > > > > > > > > configured
> > > > > > > > > > > > > > > interval(not every second, defaults to 30secs).
> > > > Storing
> > > > > > > > remote
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> log
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> metadata in a strongly consistent
> > > store
> > > > > for
> > > > > > > S3
> > > > > > > > > RSM
> > > > > > > > > > is
> > > > > > > > > > > > > > raised
> > > > > > > > > > > > > > > in PR-comment[1].
> > > > > > > > > > > > > > > > > > >>>>>>> RLM invokes RSM at regular
> > intervals
> > > > and
> > > > > RSM
> > > > > > > > can
> > > > > > > > > > give
> > > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> segment
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> metadata if it is available. RSM is
> > > > > > > responsible
> > > > > > > > > for
> > > > > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> fetching those entries. It should
> > be
> > > > > based on
> > > > > > > > > > whatever
> > > > > > > > > > > > > > > mechanism
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> consistent and efficient with the
> > > > > respective
> > > > > > > > > remote
> > > > > > > > > > > > > > storage.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Can you give more details about
> > push
> > > > > based
> > > > > > > > > > mechanism
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > RSM?
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 1.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > >
> > > > > > > > >
> > > https://github.com/apache/kafka/pull/7561#discussion_r344576223
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > >>>>>>> Satish.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Dec 5, 2019 at 4:23 AM Jun
> > > Rao
> > > > <
> > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi, Harsha,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 40/41. I am curious which block
> > > > storages
> > > > > you
> > > > > > > > have
> > > > > > > > > > > > tested.
> > > > > > > > > > > > > > S3
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> seems
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to be
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> one of the popular block stores.
> > The
> > > > > concerns
> > > > > > > > > that
> > > > > > > > > > I
> > > > > > > > > > > > have
> > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> pull
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> based
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> approach are the following.
> > > > > > > > > > > > > > > > > > >>>>>>> (a) Cost: S3 list object requests
> > > cost
> > > > > $0.005
> > > > > > > > per
> > > > > > > > > > 1000
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> requests. If
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> have 100,000 partitions and want to
> > > > pull
> > > > > the
> > > > > > > > > > metadata
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> partition
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> at
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the rate of 1/sec. It can cost
> > > > $0.5/sec,
> > > > > > > which
> > > > > > > > is
> > > > > > > > > > > > roughly
> > > > > > > > > > > > > > > $40K
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> per
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> day.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (b) Semantics: S3 list objects are
> > > > > eventually
> > > > > > > > > > > > consistent.
> > > > > > > > > > > > > > So,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> when
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> do a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> list object request, there is no
> > > > > guarantee
> > > > > > > that
> > > > > > > > > > you can
> > > > > > > > > > > > > see
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> uploaded
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> objects. This could impact the
> > > > > correctness of
> > > > > > > > > > > > subsequent
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> logics.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (c) Efficiency: Blindly pulling
> > > > metadata
> > > > > when
> > > > > > > > > > there is
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> change adds
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> unnecessary overhead in the broker
> > as
> > > > > well as
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > block
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> store.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> So, have you guys tested S3? If so,
> > > > > could you
> > > > > > > > > share
> > > > > > > > > > > > your
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> experience
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> terms of cost, semantics and
> > > > efficiency?
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Dec 3, 2019 at 10:11 PM
> > > Harsha
> > > > > > > > > > Chintalapani <
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> kafka@harsha.io
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 26, 2019 at 3:46 PM,
> > Jun
> > > > Rao
> > > > > <
> > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Ying,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks for the reply.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 40/41. There are two different ways
> > > > that
> > > > > we
> > > > > > > can
> > > > > > > > > > > > approach
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> this.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> One is
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> what
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> you said. We can have an
> > opinionated
> > > > way
> > > > > of
> > > > > > > > > > storing and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> populating
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> metadata that we think is good
> > enough
> > > > for
> > > > > > > > > > everyone. I
> > > > > > > > > > > > am
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> sure if
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> is the case based on what's
> > currently
> > > > > > > proposed
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> For
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> example, I
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> am not sure that (1) everyone
> > always
> > > > > needs
> > > > > > > > local
> > > > > > > > > > > > > metadata;
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (2)
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> current
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> local storage format is general
> > > enough
> > > > > and
> > > > > > > (3)
> > > > > > > > > > everyone
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wants to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> use
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> pull based approach to propagate
> > the
> > > > > > > metadata.
> > > > > > > > > > Another
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> approach
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> is to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> make
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> this pluggable and let the
> > > implementor
> > > > > > > > implements
> > > > > > > > > > the
> > > > > > > > > > > > > best
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> approach
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> for a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> particular block storage. I haven't
> > > > seen
> > > > > any
> > > > > > > > > > comments
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Slack/AirBnb
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the mailing list on this topic. It
> > > > would
> > > > > be
> > > > > > > > great
> > > > > > > > > > if
> > > > > > > > > > > > they
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> provide
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> feedback directly here.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> The current interfaces are designed
> > > > with
> > > > > most
> > > > > > > > > > popular
> > > > > > > > > > > > > block
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storages
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> available today and we did 2
> > > > > implementations
> > > > > > > > with
> > > > > > > > > > these
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> interfaces and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> they both are yielding good results
> > > as
> > > > we
> > > > > > > going
> > > > > > > > > > through
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> testing of
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> it.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> If there is ever a need for pull
> > > based
> > > > > > > approach
> > > > > > > > > we
> > > > > > > > > > can
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> definitely
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> evolve
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the interface.
> > > > > > > > > > > > > > > > > > >>>>>>> In the past we did mark interfaces
> > to
> > > > be
> > > > > > > > evolving
> > > > > > > > > > to
> > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> room for
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> unknowns
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> in the future.
> > > > > > > > > > > > > > > > > > >>>>>>> If you have any suggestions around
> > > the
> > > > > > > current
> > > > > > > > > > > > interfaces
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> please
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> propose we
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> are happy to see if we can work
> > them
> > > > > into it.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 43. To offer tier storage as a
> > > general
> > > > > > > feature,
> > > > > > > > > > ideally
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> existing
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> capabilities should still be
> > > supported.
> > > > > It's
> > > > > > > > fine
> > > > > > > > > > if
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> uber
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> implementation doesn't support all
> > > > > > > capabilities
> > > > > > > > > for
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> internal
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> usage.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> However, the framework should be
> > > > general
> > > > > > > > enough.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> We agree on that as a principle.
> > But
> > > > all
> > > > > of
> > > > > > > > these
> > > > > > > > > > major
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> features
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> mostly
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> coming right now and to have a new
> > > big
> > > > > > > feature
> > > > > > > > > > such as
> > > > > > > > > > > > > > tiered
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> support all the new features will
> > be
> > > a
> > > > > big
> > > > > > > ask.
> > > > > > > > > We
> > > > > > > > > > can
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> document on
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> how
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> do
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> we approach solving these in future
> > > > > > > iterations.
> > > > > > > > > > > > > > > > > > >>>>>>> Our goal is to make this tiered
> > > storage
> > > > > > > feature
> > > > > > > > > > work
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> everyone.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 43.3 This is more than just serving
> > > the
> > > > > > > tier-ed
> > > > > > > > > > data
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> block
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> With KIP-392, the consumer now can
> > > > > resolve
> > > > > > > the
> > > > > > > > > > > > conflicts
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> with the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> replica
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> based on leader epoch. So, we need
> > to
> > > > > make
> > > > > > > sure
> > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> leader epoch
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> can be
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> recovered properly from tier
> > storage.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> We are working on testing our
> > > approach
> > > > > and we
> > > > > > > > > will
> > > > > > > > > > > > update
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the KIP
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> design details.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 43.4 For JBOD, if tier storage
> > stores
> > > > the
> > > > > > > tier
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> locally, we
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> need to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> support moving such metadata across
> > > > disk
> > > > > > > > > > directories
> > > > > > > > > > > > > since
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> JBOD
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> supports
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> moving data across disks.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> KIP is updated with JBOD details.
> > > > Having
> > > > > said
> > > > > > > > > that
> > > > > > > > > > JBOD
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> tooling
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> needs
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> evolve to support production loads.
> > > > Most
> > > > > of
> > > > > > > the
> > > > > > > > > > users
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> interested in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> using tiered storage without JBOD
> > > > support
> > > > > > > > support
> > > > > > > > > > on
> > > > > > > > > > > > day
> > > > > > > > > > > > > 1.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > >>>>>>> Harsha
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> As for meeting, we could have a KIP
> > > > > e-meeting
> > > > > > > > on
> > > > > > > > > > this
> > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> needed,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> but it
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> will be open to everyone and will
> > be
> > > > > recorded
> > > > > > > > and
> > > > > > > > > > > > shared.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Often,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> details are still resolved through
> > > the
> > > > > > > mailing
> > > > > > > > > > list.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 19, 2019 at 6:48 PM
> > Ying
> > > > > Zheng
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> <yi...@uber.com.invalid>
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Please ignore my previous email
> > > > > > > > > > > > > > > > > > >>>>>>> I didn't know Apache requires all
> > the
> > > > > > > > discussions
> > > > > > > > > > to be
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> "open"
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Tue, Nov 19, 2019, 5:40 PM Ying
> > > > Zheng
> > > > > <
> > > > > > > > > > > > yingz@uber.com
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thank you very much for your
> > > feedback!
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Can we schedule a meeting in your
> > > Palo
> > > > > Alto
> > > > > > > > > office
> > > > > > > > > > in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> December? I
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> think a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> face to face discussion is much
> > more
> > > > > > > efficient
> > > > > > > > > than
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> emails. Both
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Harsha
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> I can visit you. Satish may be able
> > > to
> > > > > join
> > > > > > > us
> > > > > > > > > > > > remotely.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Nov 15, 2019 at 11:04 AM
> > Jun
> > > > Rao
> > > > > <
> > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Hi, Satish and Harsha,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> The following is a more detailed
> > high
> > > > > level
> > > > > > > > > > feedback
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the KIP.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Overall,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the KIP seems useful. The challenge
> > > is
> > > > > how to
> > > > > > > > > > design it
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> such that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> it’s
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> general enough to support different
> > > > ways
> > > > > of
> > > > > > > > > > > > implementing
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> feature
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> support existing features.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 40. Local segment metadata storage:
> > > The
> > > > > KIP
> > > > > > > > makes
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> assumption
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> metadata for the archived log
> > > segments
> > > > > are
> > > > > > > > cached
> > > > > > > > > > > > locally
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> every
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> broker
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> and provides a specific
> > > implementation
> > > > > for
> > > > > > > the
> > > > > > > > > > local
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> framework. We probably should
> > discuss
> > > > > this
> > > > > > > > more.
> > > > > > > > > > For
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> example,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> some
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage providers may not want to
> > > cache
> > > > > the
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> locally and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> just
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> rely
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> upon a remote key/value store if
> > > such a
> > > > > store
> > > > > > > > is
> > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> present. If
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> local store is used, there could be
> > > > > different
> > > > > > > > > ways
> > > > > > > > > > of
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> implementing it
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (e.g., based on customized local
> > > files,
> > > > > an
> > > > > > > > > embedded
> > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> store
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> like
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> RocksDB, etc). An alternative of
> > > > > designing
> > > > > > > this
> > > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> provide an
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> interface for retrieving the tier
> > > > segment
> > > > > > > > > metadata
> > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> leave the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> details
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> of
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> how to get the metadata outside of
> > > the
> > > > > > > > framework.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 41. RemoteStorageManager interface
> > > and
> > > > > the
> > > > > > > > usage
> > > > > > > > > > of the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> interface in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> framework: I am not sure if the
> > > > > interface is
> > > > > > > > > > general
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> enough. For
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> example,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> it seems that RemoteLogIndexEntry
> > is
> > > > > tied to
> > > > > > > a
> > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> way of
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storing
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> metadata in remote storage. The
> > > > framework
> > > > > > > uses
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> listRemoteSegments()
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> api
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> a pull based approach. However, in
> > > some
> > > > > other
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> implementations, a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> push
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> based
> > > > > > > > > > > > > > > > > > >>>>>>> approach may be more preferred. I
> > > don’t
> > > > > have
> > > > > > > a
> > > > > > > > > > concrete
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> proposal
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> yet.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> But,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> it would be useful to give this
> > area
> > > > some
> > > > > > > more
> > > > > > > > > > thoughts
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> and see
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> if we
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> make the interface more general.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 42. In the diagram, the
> > > > RemoteLogManager
> > > > > is
> > > > > > > > side
> > > > > > > > > by
> > > > > > > > > > > > side
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> LogManager.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> This KIP only discussed how the
> > fetch
> > > > > request
> > > > > > > > is
> > > > > > > > > > > > handled
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> between
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> two
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> layer. However, we should also
> > > consider
> > > > > how
> > > > > > > > other
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> touch
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> log can be handled. e.g., list
> > > offsets
> > > > by
> > > > > > > > > > timestamp,
> > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> records,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> etc.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Also, in this model, it's not clear
> > > > which
> > > > > > > > > > component is
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> responsible
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> for
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> managing the log start offset. It
> > > seems
> > > > > that
> > > > > > > > the
> > > > > > > > > > log
> > > > > > > > > > > > > start
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> offset
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> could
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> be
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> changed by both RemoteLogManager
> > and
> > > > > > > > LogManager.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 43. There are quite a few existing
> > > > > features
> > > > > > > not
> > > > > > > > > > covered
> > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> KIP.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> It
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> would be useful to discuss each of
> > > > those.
> > > > > > > > > > > > > > > > > > >>>>>>> 43.1 I won’t say that compacted
> > > topics
> > > > > are
> > > > > > > > rarely
> > > > > > > > > > used
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> always
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> small.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> For example, KStreams uses
> > compacted
> > > > > topics
> > > > > > > for
> > > > > > > > > > storing
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> states
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> and
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> sometimes the size of the topic
> > could
> > > > be
> > > > > > > large.
> > > > > > > > > > While
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> might
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> be ok
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> support compacted topics initially,
> > > it
> > > > > would
> > > > > > > be
> > > > > > > > > > useful
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> have a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> high
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> level
> > > > > > > > > > > > > > > > > > >>>>>>> idea on how this might be supported
> > > > down
> > > > > the
> > > > > > > > road
> > > > > > > > > > so
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> we
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> don’t
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> have
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> make incompatible API changes in
> > the
> > > > > future.
> > > > > > > > > > > > > > > > > > >>>>>>> 43.2 We need to discuss how EOS is
> > > > > supported.
> > > > > > > > In
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> particular, how
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> producer state integrated with the
> > > > remote
> > > > > > > > > storage.
> > > > > > > > > > 43.3
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Now that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> KIP-392
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (allow consumers to fetch from
> > > closest
> > > > > > > replica)
> > > > > > > > > is
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> implemented,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> we
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> need
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> discuss how reading from a follower
> > > > > replica
> > > > > > > is
> > > > > > > > > > > > supported
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> tier
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> 43.4 We need to discuss how JBOD is
> > > > > supported
> > > > > > > > > with
> > > > > > > > > > tier
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Jun
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Fri, Nov 8, 2019 at 12:06 AM Tom
> > > > > Bentley <
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> tbentley@redhat.com
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks for those insights Ying.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> On Thu, Nov 7, 2019 at 9:26 PM Ying
> > > > Zheng
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> <yingz@uber.com.invalid
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> wrote:
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Thanks, I missed that point.
> > However,
> > > > > there's
> > > > > > > > > > still a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> point at
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> which
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> consumer fetches start getting
> > served
> > > > > from
> > > > > > > > remote
> > > > > > > > > > > > storage
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (even
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> if
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> point isn't as soon as the local
> > log
> > > > > > > retention
> > > > > > > > > > > > > time/size).
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> This
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> represents
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> a kind of performance cliff edge
> > and
> > > > > what I'm
> > > > > > > > > > really
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> interested
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> in
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> how
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> easy it is for a consumer which
> > falls
> > > > off
> > > > > > > that
> > > > > > > > > > cliff to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> catch up
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> and so
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> its
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> fetches again come from local
> > > storage.
> > > > > > > > Obviously
> > > > > > > > > > this
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> depend
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> on
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> all
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> sorts of factors (like production
> > > rate,
> > > > > > > > > consumption
> > > > > > > > > > > > > rate),
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> so
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> it's
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> not
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> guaranteed (just like it's not
> > > > > guaranteed for
> > > > > > > > > Kafka
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> today), but
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> this
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> would
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> represent a new failure mode.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> As I have explained in the last
> > mail,
> > > > > it's a
> > > > > > > > very
> > > > > > > > > > rare
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> case that
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> a
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> consumer
> > > > > > > > > > > > > > > > > > >>>>>>> need to read remote data. With our
> > > > > experience
> > > > > > > > at
> > > > > > > > > > Uber,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> this only
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> happens
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> when the consumer service had an
> > > outage
> > > > > for
> > > > > > > > > several
> > > > > > > > > > > > > hours.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> There is not a "performance cliff"
> > as
> > > > you
> > > > > > > > assume.
> > > > > > > > > > The
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> is
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> even faster than local disks in
> > terms
> > > > of
> > > > > > > > > bandwidth.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Reading from
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage is going to have higher
> > > latency
> > > > > than
> > > > > > > > > local
> > > > > > > > > > > > disk.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> But
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> since
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> consumer
> > > > > > > > > > > > > > > > > > >>>>>>> is catching up several hours data,
> > > it's
> > > > > not
> > > > > > > > > > sensitive
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> sub-second
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> level
> > > > > > > > > > > > > > > > > > >>>>>>> latency, and each remote read
> > request
> > > > > will
> > > > > > > > read a
> > > > > > > > > > large
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> amount of
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> data to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> make the overall performance better
> > > > than
> > > > > > > > reading
> > > > > > > > > > from
> > > > > > > > > > > > > local
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> disks.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Another aspect I'd like to
> > understand
> > > > > better
> > > > > > > is
> > > > > > > > > the
> > > > > > > > > > > > > effect
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> of
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> serving
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> fetch
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> request from remote storage has on
> > > the
> > > > > > > broker's
> > > > > > > > > > network
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> utilization. If
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> we're just trimming the amount of
> > > data
> > > > > held
> > > > > > > > > locally
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> (without
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> increasing
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> overall local+remote retention),
> > then
> > > > > we're
> > > > > > > > > > effectively
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> trading
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> disk
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> bandwidth for network bandwidth
> > when
> > > > > serving
> > > > > > > > > fetch
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> requests from
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> remote
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage (which I understand to be a
> > > > good
> > > > > > > thing,
> > > > > > > > > > since
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> brokers are
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> often/usually disk bound). But if
> > > we're
> > > > > > > > > increasing
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> overall
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> local+remote
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> retention then it's more likely
> > that
> > > > > network
> > > > > > > > > itself
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> becomes the
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> bottleneck.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> I appreciate this is all rather
> > hand
> > > > > wavy,
> > > > > > > I'm
> > > > > > > > > just
> > > > > > > > > > > > > trying
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> understand
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> how this would affect broker
> > > > > performance, so
> > > > > > > > I'd
> > > > > > > > > be
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> grateful for
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> any
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> insights you can offer.
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> Network bandwidth is a function of
> > > > > produce
> > > > > > > > speed,
> > > > > > > > > > it
> > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> nothing
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> to
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> do
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> with
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> remote retention. As long as the
> > data
> > > > is
> > > > > > > > shipped
> > > > > > > > > to
> > > > > > > > > > > > > remote
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> storage,
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> you
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> can
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> keep the data there for 1 day or 1
> > > year
> > > > > or
> > > > > > > 100
> > > > > > > > > > years,
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> doesn't
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> consume
> > > > > > > > > > > > > > > > > > >>>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>> any
> > > > > > > > > > > > > > > > > > >>>>>>> network resources.
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > > > > > >>>>>>
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >