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 <j...@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.dugg...@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.dugg...@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 <j...@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 <
> > kpraka...@confluent.io>
> > > > wrote:
> > > >
> > > > > Hi Satish,
> > > > >
> > > > > Thanks for the updates to the KIP. Here are my first batch of
> > > > > comments/suggestions on the latest version of the KIP.
> > > > >
> > > > > 5012. In the RemoteStorageManager interface, there is an API defined
> > for
> > > > > each file type. For example, fetchOffsetIndex, fetchTimestampIndex
> > etc. To
> > > > > avoid the duplication, I'd suggest we can instead have a FileType
> > enum and
> > > > > a common get API based on the FileType.
> > > > >
> > > > > 5013. There are some references to the Google doc in the KIP. I
> > wasn't sure
> > > > > if the Google doc is expected to be in sync with the contents of the
> > wiki.
> > > > > Going forward, it seems easier if just the KIP is maintained as the
> > source
> > > > > of truth. In this regard, could you please move all the references
> > to the
> > > > > Google doc, maybe to a separate References section at the bottom of
> > the
> > > > > KIP?
> > > > >
> > > > > 5014. There are some TODO sections in the KIP. Would these be filled
> > up in
> > > > > future iterations?
> > > > >
> > > > > 5015. Under "Topic deletion lifecycle", I'm trying to understand why
> > do we
> > > > > need delete_partition_marked as well as the delete_partition_started
> > > > > messages. I couldn't spot a drawback if supposing we simplified the
> > design
> > > > > such that the controller would only write delete_partition_started
> > message,
> > > > > and RemoteLogCleaner (RLC) instance picks it up for processing. What
> > am I
> > > > > missing?
> > > > >
> > > > > 5016. Under "Topic deletion lifecycle", step (4) is mentioned as
> > "RLC gets
> > > > > all the remote log segments for the partition and each of these
> > remote log
> > > > > segments is deleted with the next steps.". Since the RLC instance
> > runs on
> > > > > each tier topic partition leader, how does the RLC then get the list
> > of
> > > > > remote log segments to be deleted? It will be useful to add that
> > detail to
> > > > > the KIP.
> > > > >
> > > > > 5017. Under "Public Interfaces -> Configs", there is a line
> > mentioning "We
> > > > > will support flipping remote.log.storage.enable in next versions."
> > It will
> > > > > be useful to mention this in the "Future Work" section of the KIP
> > too.
> > > > >
> > > > > 5018. The KIP introduces a number of configuration parameters. It
> > will be
> > > > > useful to mention in the KIP if the user should assume these as
> > static
> > > > > configuration in the server.properties file, or dynamic
> > configuration which
> > > > > can be modified without restarting the broker.
> > > > >
> > > > > 5019.  Maybe this is planned as a future update to the KIP, but I
> > thought
> > > > > I'd mention it here. Could you please add details to the KIP on why
> > RocksDB
> > > > > was chosen as the default cache implementation of RLMM, and how it
> > is going
> > > > > to be used? Were alternatives compared/considered? For example, it
> > would be
> > > > > useful to explain/evaluate the following: 1) debuggability of the
> > RocksDB
> > > > > JNI interface, 2) performance, 3) portability across platforms and 4)
> > > > > interface parity of RocksDB’s JNI api with it's underlying C/C++ api.
> > > > >
> > > > > 5020. Following up on (5019), for the RocksDB cache, it will be
> > useful to
> > > > > explain the relationship/mapping between the following in the KIP:
> > 1) # of
> > > > > tiered partitions, 2) # of partitions of metadata topic
> > > > > __remote_log_metadata and 3) # of RocksDB instances. i.e. is the
> > plan to
> > > > > have a RocksDB instance per tiered partition, or per metadata topic
> > > > > partition, or just 1 for per broker?
> > > > >
> > > > > 5021. I was looking at the implementation prototype (PR link:
> > > > > https://github.com/apache/kafka/pull/7561). It seems that a boolean
> > > > > attribute is being introduced into the Log layer to check if remote
> > log
> > > > > capability is enabled. While the boolean footprint is small at the
> > moment,
> > > > > this can easily grow in the future and become harder to
> > > > > test/maintain, considering that the Log layer is already pretty
> > complex. We
> > > > > should start thinking about how to manage such changes to the Log
> > layer
> > > > > (for the purpose of improved testability, better separation of
> > concerns and
> > > > > readability). One proposal I have is to take a step back and define a
> > > > > higher level Log interface. Then, the Broker code can be changed to
> > use
> > > > > this interface. It can be changed such that only a handle to the
> > interface
> > > > > is exposed to other components (such as LogCleaner, ReplicaManager
> > etc.)
> > > > > and not the underlying Log object. This approach keeps the user of
> > the Log
> > > > > layer agnostic of the whereabouts of the data. Underneath the
> > interface,
> > > > > the implementing classes can completely separate local log
> > capabilities
> > > > > from the remote log. For example, the Log class can be simplified to
> > only
> > > > > manage logic surrounding local log segments and metadata.
> > Additionally, a
> > > > > wrapper class can be provided (implementing the higher level Log
> > interface)
> > > > > which will contain any/all logic surrounding tiered data. The wrapper
> > > > > class will wrap around an instance of the Log class delegating the
> > local
> > > > > log logic to it. Finally, a handle to the wrapper class can be
> > exposed to
> > > > > the other components wherever they need a handle to the higher level
> > Log
> > > > > interface.
> > > > >
> > > > >
> > > > > Cheers,
> > > > > Kowshik
> > > > >
> > > > > On Mon, Oct 26, 2020 at 9:52 PM Satish Duggana <
> > satish.dugg...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi,
> > > > > > KIP is updated with 1) topic deletion lifecycle and its related
> > items
> > > > > > 2) Protocol changes(mainly related to ListOffsets) and other minor
> > > > > > changes.
> > > > > > Please go through them and let us know your comments.
> > > > > >
> > > > > > Thanks,
> > > > > > Satish.
> > > > > >
> > > > > > On Mon, Sep 28, 2020 at 9:10 PM Satish Duggana <
> > satish.dugg...@gmail.com
> > > > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > Hi Dhruvil,
> > > > > > > Thanks for looking into the KIP and sending your comments. Sorry
> > for
> > > > > > > the late reply, missed it in the mail thread.
> > > > > > >
> > > > > > > 1. Could you describe how retention would work with this KIP and
> > which
> > > > > > > threads are responsible for driving this work? I believe there
> > are 3
> > > > > > kinds
> > > > > > > of retention processes we are looking at:
> > > > > > >   (a) Regular retention for data in tiered storage as per
> > configured `
> > > > > > > retention.ms` / `retention.bytes`.
> > > > > > >   (b) Local retention for data in local storage as per
> > configured `
> > > > > > > local.log.retention.ms` / `local.log.retention.bytes`
> > > > > > >   (c) Possibly regular retention for data in local storage, if
> > the
> > > > > > tiering
> > > > > > > task is lagging or for data that is below the log start offset.
> > > > > > >
> > > > > > > Local log retention is done by the existing log cleanup tasks.
> > These
> > > > > > > are not done for segments that are not yet copied to remote
> > storage.
> > > > > > > Remote log cleanup is done by the leader partition’s RLMTask.
> > > > > > >
> > > > > > > 2. When does a segment become eligible to be tiered? Is it as
> > soon as
> > > > > the
> > > > > > > segment is rolled and the end offset is less than the last stable
> > > > > offset
> > > > > > as
> > > > > > > mentioned in the KIP? I wonder if we need to consider other
> > parameters
> > > > > > too,
> > > > > > > like the highwatermark so that we are guaranteed that what we are
> > > > > tiering
> > > > > > > has been committed to the log and accepted by the ISR.
> > > > > > >
> > > > > > > AFAIK, last stable offset is always <= highwatermark. This will
> > make
> > > > > > > sure we are always tiering the message segments which have been
> > > > > > > accepted by ISR and transactionally completed.
> > > > > > >
> > > > > > >
> > > > > > > 3. The section on "Follower Fetch Scenarios" is useful but is a
> > bit
> > > > > > > difficult to parse at the moment. It would be useful to
> > summarize the
> > > > > > > changes we need in the ReplicaFetcher.
> > > > > > >
> > > > > > > It may become difficult for users to read/follow if we add code
> > changes
> > > > > > here.
> > > > > > >
> > > > > > > 4. Related to the above, it's a bit unclear how we are planning
> > on
> > > > > > > restoring the producer state for a new replica. Could you expand
> > on
> > > > > that?
> > > > > > >
> > > > > > > It is mentioned in the KIP BuildingRemoteLogAuxState is
> > introduced to
> > > > > > > build the state like leader epoch sequence and producer snapshots
> > > > > > > before it starts fetching the data from the leader. We will make
> > it
> > > > > > > clear in the KIP.
> > > > > > >
> > > > > > >
> > > > > > > 5. Similarly, it would be worth summarizing the behavior on
> > unclean
> > > > > > leader
> > > > > > > election. There are several scenarios to consider here: data
> > loss from
> > > > > > > local log, data loss from remote log, data loss from metadata
> > topic,
> > > > > etc.
> > > > > > > It's worth describing these in detail.
> > > > > > >
> > > > > > > We mentioned the cases about unclean leader election in the
> > follower
> > > > > > > fetch scenarios.
> > > > > > > If there are errors while fetching data from remote store or
> > metadata
> > > > > > > store, it will work the same way as it works with local log. It
> > > > > > > returns the error back to the caller. Please let us know if I am
> > > > > > > missing your point here.
> > > > > > >
> > > > > > >
> > > > > > > 7. For a READ_COMMITTED FetchRequest, how do we retrieve and
> > return the
> > > > > > > aborted transaction metadata?
> > > > > > >
> > > > > > > When a fetch for a remote log is accessed, we will fetch aborted
> > > > > > > transactions along with the segment if it is not found in the
> > local
> > > > > > > index cache. This includes the case of transaction index not
> > existing
> > > > > > > in the remote log segment. That means, the cache entry can be
> > empty or
> > > > > > > have a list of aborted transactions.
> > > > > > >
> > > > > > >
> > > > > > > 8. The `LogSegmentData` class assumes that we have a log segment,
> > > > > offset
> > > > > > > index, time index, transaction index, producer snapshot and
> > leader
> > > > > epoch
> > > > > > > index. How do we deal with cases where we do not have one or
> > more of
> > > > > > these?
> > > > > > > For example, we may not have a transaction index or producer
> > snapshot
> > > > > > for a
> > > > > > > particular segment. The former is optional, and the latter is
> > only kept
> > > > > > for
> > > > > > > up to the 3 latest segments.
> > > > > > >
> > > > > > > This is a good point,  we discussed this in the last meeting.
> > > > > > > Transaction index is optional and we will copy them only if it
> > exists.
> > > > > > > We want to keep all the producer snapshots at each log segment
> > rolling
> > > > > > > and they can be removed if the log copying is successful and it
> > still
> > > > > > > maintains the existing latest 3 segments, We only delete the
> > producer
> > > > > > > snapshots which have been copied to remote log segments on
> > leader.
> > > > > > > Follower will keep the log segments beyond the segments which
> > have not
> > > > > > > been copied to remote storage. We will update the KIP with these
> > > > > > > details.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Thu, Sep 17, 2020 at 1:47 AM Dhruvil Shah <
> > dhru...@confluent.io>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > Hi Satish, Harsha,
> > > > > > > >
> > > > > > > > Thanks for the KIP. Few questions below:
> > > > > > > >
> > > > > > > > 1. Could you describe how retention would work with this KIP
> > and
> > > > > which
> > > > > > > > threads are responsible for driving this work? I believe there
> > are 3
> > > > > > kinds
> > > > > > > > of retention processes we are looking at:
> > > > > > > >   (a) Regular retention for data in tiered storage as per
> > configured
> > > > > `
> > > > > > > > retention.ms` / `retention.bytes`.
> > > > > > > >   (b) Local retention for data in local storage as per
> > configured `
> > > > > > > > local.log.retention.ms` / `local.log.retention.bytes`
> > > > > > > >   (c) Possibly regular retention for data in local storage, if
> > the
> > > > > > tiering
> > > > > > > > task is lagging or for data that is below the log start offset.
> > > > > > > >
> > > > > > > > 2. When does a segment become eligible to be tiered? Is it as
> > soon as
> > > > > > the
> > > > > > > > segment is rolled and the end offset is less than the last
> > stable
> > > > > > offset as
> > > > > > > > mentioned in the KIP? I wonder if we need to consider other
> > > > > parameters
> > > > > > too,
> > > > > > > > like the highwatermark so that we are guaranteed that what we
> > are
> > > > > > tiering
> > > > > > > > has been committed to the log and accepted by the ISR.
> > > > > > > >
> > > > > > > > 3. The section on "Follower Fetch Scenarios" is useful but is
> > a bit
> > > > > > > > difficult to parse at the moment. It would be useful to
> > summarize the
> > > > > > > > changes we need in the ReplicaFetcher.
> > > > > > > >
> > > > > > > > 4. Related to the above, it's a bit unclear how we are
> > planning on
> > > > > > > > restoring the producer state for a new replica. Could you
> > expand on
> > > > > > that?
> > > > > > > >
> > > > > > > > 5. Similarly, it would be worth summarizing the behavior on
> > unclean
> > > > > > leader
> > > > > > > > election. There are several scenarios to consider here: data
> > loss
> > > > > from
> > > > > > > > local log, data loss from remote log, data loss from metadata
> > topic,
> > > > > > etc.
> > > > > > > > It's worth describing these in detail.
> > > > > > > >
> > > > > > > > 6. It would be useful to add details about how we plan on using
> > > > > > RocksDB in
> > > > > > > > the default implementation of `RemoteLogMetadataManager`.
> > > > > > > >
> > > > > > > > 7. For a READ_COMMITTED FetchRequest, how do we retrieve and
> > return
> > > > > the
> > > > > > > > aborted transaction metadata?
> > > > > > > >
> > > > > > > > 8. The `LogSegmentData` class assumes that we have a log
> > segment,
> > > > > > offset
> > > > > > > > index, time index, transaction index, producer snapshot and
> > leader
> > > > > > epoch
> > > > > > > > index. How do we deal with cases where we do not have one or
> > more of
> > > > > > these?
> > > > > > > > For example, we may not have a transaction index or producer
> > snapshot
> > > > > > for a
> > > > > > > > particular segment. The former is optional, and the latter is
> > only
> > > > > > kept for
> > > > > > > > up to the 3 latest segments.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dhruvil
> > > > > > > >
> > > > > > > > On Mon, Sep 7, 2020 at 6:54 PM Harsha Ch <harsha...@gmail.com>
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi All,
> > > > > > > > >
> > > > > > > > > We are all working through the last meeting feedback. I'll
> > cancel
> > > > > the
> > > > > > > > > tomorrow 's meeting and we can meanwhile continue our
> > discussion in
> > > > > > mailing
> > > > > > > > > list. We can start the regular meeting from next week
> > onwards.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Harsha
> > > > > > > > >
> > > > > > > > > On Fri, Sep 04, 2020 at 8:41 AM, Satish Duggana <
> > > > > > satish.dugg...@gmail.com
> > > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Hi Jun,
> > > > > > > > > > Thanks for your thorough review and comments. Please find
> > the
> > > > > > inline
> > > > > > > > > > replies below.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 600. The topic deletion logic needs more details.
> > > > > > > > > > 600.1 The KIP mentions "The controller considers the topic
> > > > > > partition is
> > > > > > > > > > deleted only when it determines that there are no log
> > segments
> > > > > for
> > > > > > that
> > > > > > > > > > topic partition by using RLMM". How is this done?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > It uses RLMM#listSegments() returns all the segments for
> > the
> > > > > given
> > > > > > topic
> > > > > > > > > > partition.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 600.2 "If the delete option is enabled then the leader
> > will stop
> > > > > > RLM task
> > > > > > > > > > and stop processing and it sets all the remote log segment
> > > > > > metadata of
> > > > > > > > > > that partition with a delete marker and publishes them to
> > RLMM."
> > > > > We
> > > > > > > > > > discussed this earlier. When a topic is being deleted,
> > there may
> > > > > > not be a
> > > > > > > > > > leader for the deleted partition.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > This is a good point. As suggested in the meeting, we will
> > add a
> > > > > > separate
> > > > > > > > > > section for topic/partition deletion lifecycle and this
> > scenario
> > > > > > will be
> > > > > > > > > > addressed.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 601. Unclean leader election
> > > > > > > > > > 601.1 Scenario 1: new empty follower
> > > > > > > > > > After step 1, the follower restores up to offset 3. So why
> > does
> > > > > it
> > > > > > have
> > > > > > > > > > LE-2 <https://issues.apache.org/jira/browse/LE-2> at
> > offset 5?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Nice catch. It was showing the leader epoch fetched from
> > the
> > > > > remote
> > > > > > > > > > storage. It should be shown with the truncated till offset
> > 3.
> > > > > > Updated the
> > > > > > > > > > KIP.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 601.2 senario 5: After Step 3, leader A has inconsistent
> > data
> > > > > > between its
> > > > > > > > > > local and the tiered data. For example. offset 3 has msg 3
> > LE-0
> > > > > > <https://issues.apache.org/jira/browse/LE-0> locally,
> > > > > > > > > > but msg 5 LE-1 <https://issues.apache.org/jira/browse/LE-1>
> > in
> > > > > > the remote store. While it's ok for the unclean leader
> > > > > > > > > > to lose data, it should still return consistent data,
> > whether
> > > > > it's
> > > > > > from
> > > > > > > > > > the local or the remote store.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > There is no inconsistency here as LE-0
> > > > > > <https://issues.apache.org/jira/browse/LE-0> offsets are [0, 4]
> > and LE-2
> > > > > > <https://issues.apache.org/jira/browse/LE-2>:
> > > > > > > > > > [5, ]. It will always get the right records for the given
> > offset
> > > > > > and
> > > > > > > > > > leader epoch. In case of remote, RSM is invoked to get the
> > remote
> > > > > > log
> > > > > > > > > > segment that contains the given offset with the leader
> > epoch.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 601.4 It seems that retention is based on
> > > > > > > > > > listRemoteLogSegments(TopicPartition topicPartition, long
> > > > > > leaderEpoch).
> > > > > > > > > > When there is an unclean leader election, it's possible
> > for the
> > > > > new
> > > > > > > > > leader
> > > > > > > > > > to not to include certain epochs in its epoch cache. How
> > are
> > > > > remote
> > > > > > > > > > segments associated with those epochs being cleaned?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > That is a good point. This leader will also cleanup the
> > epochs
> > > > > > earlier to
> > > > > > > > > > its start leader epoch and delete those segments. It gets
> > the
> > > > > > earliest
> > > > > > > > > > epoch for a partition and starts deleting segments from
> > that
> > > > > leader
> > > > > > > > > epoch.
> > > > > > > > > > We need one more API in RLMM to get the earliest leader
> > epoch.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 601.5 The KIP discusses the handling of unclean leader
> > elections
> > > > > > for user
> > > > > > > > > > topics. What about unclean leader elections on
> > > > > > > > > > __remote_log_segment_metadata?
> > > > > > > > > > This is the same as other system topics like
> > consumer_offsets,
> > > > > > > > > > __transaction_state topics. As discussed in the meeting,
> > we will
> > > > > > add the
> > > > > > > > > > behavior of __remote_log_segment_metadata topic’s unclean
> > leader
> > > > > > > > > > truncation.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 602. It would be useful to clarify the limitations in the
> > initial
> > > > > > > > > release.
> > > > > > > > > > The KIP mentions not supporting compacted topics. What
> > about JBOD
> > > > > > and
> > > > > > > > > > changing the configuration of a topic from delete to
> > compact
> > > > > after
> > > > > > > > > remote.
> > > > > > > > > > log. storage. enable ( http://remote.log.storage.enable/
> > ) is
> > > > > > enabled?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > This was updated in the KIP earlier.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 603. RLM leader tasks:
> > > > > > > > > > 603.1"It checks for rolled over LogSegments (which have
> > the last
> > > > > > message
> > > > > > > > > > offset less than last stable offset of that topic
> > partition) and
> > > > > > copies
> > > > > > > > > > them along with their offset/time/transaction indexes and
> > leader
> > > > > > epoch
> > > > > > > > > > cache to the remote tier." It needs to copy the producer
> > snapshot
> > > > > > too.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Right. It copies producer snapshots too as mentioned in
> > > > > > LogSegmentData.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 603.2 "Local logs are not cleaned up till those segments
> > are
> > > > > copied
> > > > > > > > > > successfully to remote even though their retention
> > time/size is
> > > > > > reached"
> > > > > > > > > > This seems weird. If the tiering stops because the remote
> > store
> > > > > is
> > > > > > not
> > > > > > > > > > available, we don't want the local data to grow forever.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > It was clarified in the discussion that the comment was
> > more
> > > > > about
> > > > > > the
> > > > > > > > > > local storage goes beyond the log.retention. The above
> > statement
> > > > > > is about
> > > > > > > > > > local.log.retention but not for the complete
> > log.retention. When
> > > > > it
> > > > > > > > > > reaches the log.retention then it will delete the local
> > logs even
> > > > > > though
> > > > > > > > > > those are not copied to remote storage.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 604. "RLM maintains a bounded cache(possibly LRU) of the
> > index
> > > > > > files of
> > > > > > > > > > remote log segments to avoid multiple index fetches from
> > the
> > > > > remote
> > > > > > > > > > storage. These indexes can be used in the same way as local
> > > > > segment
> > > > > > > > > > indexes are used." Could you provide more details on this?
> > Are
> > > > > the
> > > > > > > > > indexes
> > > > > > > > > > cached in memory or on disk? If on disk, where are they
> > stored?
> > > > > > Are the
> > > > > > > > > > cached indexes bound by a certain size?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > These are cached on disk and stored in log.dir with a name
> > > > > > > > > > “__remote_log_index_cache”. They are bound by the total
> > size.
> > > > > This
> > > > > > will
> > > > > > > > > be
> > > > > > > > > > exposed as a user configuration,
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 605. BuildingRemoteLogAux
> > > > > > > > > > 605.1 In this section, two options are listed. Which one is
> > > > > chosen?
> > > > > > > > > > Option-2, updated the KIP.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 605.2 In option 2, it says "Build the local leader epoch
> > cache by
> > > > > > cutting
> > > > > > > > > > the leader epoch sequence received from remote storage to
> > [LSO,
> > > > > > ELO].
> > > > > > > > > (LSO
> > > > > > > > > >
> > > > > > > > > > = log start offset)." We need to do the same thing for the
> > > > > producer
> > > > > > > > > > snapshot. However, it's hard to cut the producer snapshot
> > to an
> > > > > > earlier
> > > > > > > > > > offset. Another option is to simply take the lastOffset
> > from the
> > > > > > remote
> > > > > > > > > > segment and use that as the starting fetch offset in the
> > > > > follower.
> > > > > > This
> > > > > > > > > > avoids the need for cutting.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Right, this was mentioned in the “transactional support”
> > section
> > > > > > about
> > > > > > > > > > adding these details.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 606. ListOffsets: Since we need a version bump, could you
> > > > > document
> > > > > > it
> > > > > > > > > > under a protocol change section?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Sure, we will update the KIP.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 607. "LogStartOffset of a topic can point to either of
> > local
> > > > > > segment or
> > > > > > > > > > remote segment but it is initialised and maintained in the
> > Log
> > > > > > class like
> > > > > > > > > > now. This is already maintained in `Log` class while
> > loading the
> > > > > > logs and
> > > > > > > > > > it can also be fetched from RemoteLogMetadataManager."
> > What will
> > > > > > happen
> > > > > > > > > to
> > > > > > > > > > the existing logic (e.g. log recovery) that currently
> > depends on
> > > > > > > > > > logStartOffset but assumes it's local?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > They use a field called localLogStartOffset which is the
> > local
> > > > > log
> > > > > > start
> > > > > > > > > > offset..
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 608. Handle expired remote segment: How does it pick up new
> > > > > > > > > logStartOffset
> > > > > > > > > > from deleteRecords?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Good point. This was not addressed in the KIP. Will update
> > the
> > > > > KIP
> > > > > > on how
> > > > > > > > > > the RLM task handles this scenario.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 609. RLMM message format:
> > > > > > > > > > 609.1 It includes both MaxTimestamp and EventTimestamp.
> > Where
> > > > > does
> > > > > > it get
> > > > > > > > > > both since the message in the log only contains one
> > timestamp?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > `EventTimeStamp` is the timestamp at which that segment
> > metadata
> > > > > > event is
> > > > > > > > > > generated. This is more for audits.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 609.2 If we change just the state (e.g. to
> > DELETE_STARTED), it
> > > > > > seems it's
> > > > > > > > > > wasteful to have to include all other fields not changed.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > This is a good point. We thought about incremental
> > updates. But
> > > > > we
> > > > > > want
> > > > > > > > > to
> > > > > > > > > > make sure all the events are in the expected order and take
> > > > > action
> > > > > > based
> > > > > > > > > > on the latest event. Will think through the approaches in
> > detail
> > > > > > and
> > > > > > > > > > update here.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 609.3 Could you document which process makes the following
> > > > > > transitions
> > > > > > > > > > DELETE_MARKED, DELETE_STARTED, DELETE_FINISHED?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Okay, will document more details.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 610. remote.log.reader.max.pending.tasks: "Maximum remote
> > log
> > > > > > reader
> > > > > > > > > > thread pool task queue size. If the task queue is full,
> > broker
> > > > > > will stop
> > > > > > > > > > reading remote log segments." What does the broker do if
> > the
> > > > > queue
> > > > > > is
> > > > > > > > > > full?
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > It returns an error for this topic partition.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > 611. What do we return if the request offset/epoch doesn't
> > exist
> > > > > > in the
> > > > > > > > > > following API?
> > > > > > > > > > RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > > topicPartition, long offset, int epochForOffset)
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > This returns null. But we prefer to update the return type
> > as
> > > > > > Optional
> > > > > > > > > and
> > > > > > > > > > return Empty if that does not exist.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Satish.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Sep 1, 2020 at 9:45 AM Jun Rao < jun@ confluent.
> > io (
> > > > > > > > > > j...@confluent.io ) > wrote:
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> Hi, Satish,
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> Thanks for the updated KIP. Made another pass. A few more
> > > > > comments
> > > > > > > > > below.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 600. The topic deletion logic needs more details.
> > > > > > > > > >> 600.1 The KIP mentions "The controller considers the topic
> > > > > > partition is
> > > > > > > > > >> deleted only when it determines that there are no log
> > segments
> > > > > > for that
> > > > > > > > > >> topic partition by using RLMM". How is this done? 600.2
> > "If the
> > > > > > delete
> > > > > > > > > >> option is enabled then the leader will stop RLM task and
> > stop
> > > > > > processing
> > > > > > > > > >> and it sets all the remote log segment metadata of that
> > > > > partition
> > > > > > with a
> > > > > > > > > >> delete marker and publishes them to RLMM." We discussed
> > this
> > > > > > earlier.
> > > > > > > > > When
> > > > > > > > > >> a topic is being deleted, there may not be a leader for
> > the
> > > > > > deleted
> > > > > > > > > >> partition.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 601. Unclean leader election
> > > > > > > > > >> 601.1 Scenario 1: new empty follower
> > > > > > > > > >> After step 1, the follower restores up to offset 3. So
> > why does
> > > > > > it have
> > > > > > > > > >> LE-2 <https://issues.apache.org/jira/browse/LE-2> at
> > offset 5?
> > > > > > > > > >> 601.2 senario 5: After Step 3, leader A has inconsistent
> > data
> > > > > > between
> > > > > > > > > its
> > > > > > > > > >> local and the tiered data. For example. offset 3 has msg
> > 3 LE-0
> > > > > > <https://issues.apache.org/jira/browse/LE-0> locally,
> > > > > > > > > >> but msg 5 LE-1 <
> > https://issues.apache.org/jira/browse/LE-1> in
> > > > > > the remote store. While it's ok for the unclean leader
> > > > > > > > > >> to lose data, it should still return consistent data,
> > whether
> > > > > > it's from
> > > > > > > > > >> the local or the remote store.
> > > > > > > > > >> 601.3 The follower picks up log start offset using the
> > following
> > > > > > api.
> > > > > > > > > >> Suppose that we have 3 remote segments (LE,
> > SegmentStartOffset)
> > > > > > as (2,
> > > > > > > > > >> 10),
> > > > > > > > > >> (3, 20) and (7, 15) due to an unclean leader election.
> > Using the
> > > > > > > > > following
> > > > > > > > > >> api will cause logStartOffset to go backward from 20 to
> > 15. How
> > > > > > do we
> > > > > > > > > >> prevent that?
> > > > > > > > > >> earliestLogOffset(TopicPartition topicPartition, int
> > > > > leaderEpoch)
> > > > > > 601.4
> > > > > > > > > It
> > > > > > > > > >> seems that retention is based on
> > > > > > > > > >> listRemoteLogSegments(TopicPartition topicPartition, long
> > > > > > leaderEpoch).
> > > > > > > > > >> When there is an unclean leader election, it's possible
> > for the
> > > > > > new
> > > > > > > > > leader
> > > > > > > > > >> to not to include certain epochs in its epoch cache. How
> > are
> > > > > > remote
> > > > > > > > > >> segments associated with those epochs being cleaned?
> > 601.5 The
> > > > > KIP
> > > > > > > > > >> discusses the handling of unclean leader elections for
> > user
> > > > > > topics. What
> > > > > > > > > >> about unclean leader elections on
> > > > > > > > > >> __remote_log_segment_metadata?
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 602. It would be useful to clarify the limitations in the
> > > > > initial
> > > > > > > > > release.
> > > > > > > > > >> The KIP mentions not supporting compacted topics. What
> > about
> > > > > JBOD
> > > > > > and
> > > > > > > > > >> changing the configuration of a topic from delete to
> > compact
> > > > > after
> > > > > > > > > remote.
> > > > > > > > > >> log. storage. enable ( http://remote.log.storage.enable/
> > ) is
> > > > > > enabled?
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 603. RLM leader tasks:
> > > > > > > > > >> 603.1"It checks for rolled over LogSegments (which have
> > the last
> > > > > > message
> > > > > > > > > >> offset less than last stable offset of that topic
> > partition) and
> > > > > > copies
> > > > > > > > > >> them along with their offset/time/transaction indexes and
> > leader
> > > > > > epoch
> > > > > > > > > >> cache to the remote tier." It needs to copy the producer
> > > > > snapshot
> > > > > > too.
> > > > > > > > > >> 603.2 "Local logs are not cleaned up till those segments
> > are
> > > > > > copied
> > > > > > > > > >> successfully to remote even though their retention
> > time/size is
> > > > > > reached"
> > > > > > > > > >> This seems weird. If the tiering stops because the remote
> > store
> > > > > > is not
> > > > > > > > > >> available, we don't want the local data to grow forever.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 604. "RLM maintains a bounded cache(possibly LRU) of the
> > index
> > > > > > files of
> > > > > > > > > >> remote log segments to avoid multiple index fetches from
> > the
> > > > > > remote
> > > > > > > > > >> storage. These indexes can be used in the same way as
> > local
> > > > > > segment
> > > > > > > > > >> indexes are used." Could you provide more details on
> > this? Are
> > > > > the
> > > > > > > > > indexes
> > > > > > > > > >> cached in memory or on disk? If on disk, where are they
> > stored?
> > > > > > Are the
> > > > > > > > > >> cached indexes bound by a certain size?
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 605. BuildingRemoteLogAux
> > > > > > > > > >> 605.1 In this section, two options are listed. Which one
> > is
> > > > > > chosen?
> > > > > > > > > 605.2
> > > > > > > > > >> In option 2, it says "Build the local leader epoch cache
> > by
> > > > > > cutting the
> > > > > > > > > >> leader epoch sequence received from remote storage to
> > [LSO,
> > > > > ELO].
> > > > > > (LSO
> > > > > > > > > >> = log start offset)." We need to do the same thing for the
> > > > > > producer
> > > > > > > > > >> snapshot. However, it's hard to cut the producer snapshot
> > to an
> > > > > > earlier
> > > > > > > > > >> offset. Another option is to simply take the lastOffset
> > from the
> > > > > > remote
> > > > > > > > > >> segment and use that as the starting fetch offset in the
> > > > > > follower. This
> > > > > > > > > >> avoids the need for cutting.
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 606. ListOffsets: Since we need a version bump, could you
> > > > > > document it
> > > > > > > > > >> under a protocol change section?
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 607. "LogStartOffset of a topic can point to either of
> > local
> > > > > > segment or
> > > > > > > > > >> remote segment but it is initialised and maintained in
> > the Log
> > > > > > class
> > > > > > > > > like
> > > > > > > > > >> now. This is already maintained in `Log` class while
> > loading the
> > > > > > logs
> > > > > > > > > and
> > > > > > > > > >> it can also be fetched from RemoteLogMetadataManager."
> > What will
> > > > > > happen
> > > > > > > > > to
> > > > > > > > > >> the existing logic (e.g. log recovery) that currently
> > depends on
> > > > > > > > > >> logStartOffset but assumes it's local?
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 608. Handle expired remote segment: How does it pick up
> > new
> > > > > > > > > logStartOffset
> > > > > > > > > >> from deleteRecords?
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 609. RLMM message format:
> > > > > > > > > >> 609.1 It includes both MaxTimestamp and EventTimestamp.
> > Where
> > > > > > does it
> > > > > > > > > get
> > > > > > > > > >> both since the message in the log only contains one
> > timestamp?
> > > > > > 609.2 If
> > > > > > > > > we
> > > > > > > > > >> change just the state (e.g. to DELETE_STARTED), it seems
> > it's
> > > > > > wasteful
> > > > > > > > > to
> > > > > > > > > >> have to include all other fields not changed. 609.3 Could
> > you
> > > > > > document
> > > > > > > > > >> which process makes the following transitions
> > DELETE_MARKED,
> > > > > > > > > >> DELETE_STARTED, DELETE_FINISHED?
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 610. remote.log.reader.max.pending.tasks: "Maximum remote
> > log
> > > > > > reader
> > > > > > > > > >> thread pool task queue size. If the task queue is full,
> > broker
> > > > > > will stop
> > > > > > > > > >> reading remote log segments." What does the broker do if
> > the
> > > > > > queue is
> > > > > > > > > >> full?
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> 611. What do we return if the request offset/epoch
> > doesn't exist
> > > > > > in the
> > > > > > > > > >> following API?
> > > > > > > > > >> RemoteLogSegmentMetadata
> > remoteLogSegmentMetadata(TopicPartition
> > > > > > > > > >> topicPartition, long offset, int epochForOffset)
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> Jun
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> On Mon, Aug 31, 2020 at 11:19 AM Satish Duggana < satish.
> > > > > duggana@
> > > > > > > > > gmail. com
> > > > > > > > > >> ( satish.dugg...@gmail.com ) > wrote:
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> KIP is updated with
> > > > > > > > > >>> - Remote log segment metadata topic message
> > format/schema.
> > > > > > > > > >>> - Added remote log segment metadata state transitions and
> > > > > > explained how
> > > > > > > > > >>> the deletion of segments is handled, including the case
> > of
> > > > > > partition
> > > > > > > > > >>> deletions.
> > > > > > > > > >>> - Added a few more limitations in the "Non goals"
> > section.
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> Thanks,
> > > > > > > > > >>> Satish.
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> On Thu, Aug 27, 2020 at 12:42 AM Harsha Ch < harsha. ch@
> > > > > gmail.
> > > > > > com (
> > > > > > > > > >>> harsha...@gmail.com ) > wrote:
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>> Updated the KIP with Meeting Notes section
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/
> > > > > > > > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405>
> > > > > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
> > > > > > > > > >>> (
> > > > > > > > > >>>
> > > > > > > > >
> > > > > >
> > > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes
> > > > > > > > > >>> )
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>> On Tue, Aug 25, 2020 at 1:03 PM Jun Rao < jun@
> > confluent. io
> > > > > (
> > > > > > > > > >>>> j...@confluent.io ) > wrote:
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Hi, Harsha,
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Thanks for the summary. Could you add the summary and
> > the
> > > > > > recording
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> link to
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> the last section of
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/
> > > > > > > > > Kafka+Improvement+Proposals
> > > > > > > > > >>> (
> > > > > > > > > >>>
> > > > > > > > >
> > > > > >
> > > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > >>> )
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> ?
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> Jun
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> On Tue, Aug 25, 2020 at 11:12 AM Harsha Chintalapani <
> > kafka@
> > > > > > > > > harsha. io (
> > > > > > > > > >>>>> ka...@harsha.io ) > wrote:
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> Thanks everyone for attending the meeting today.
> > > > > > > > > >>>>>> Here is the recording
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> https:/ / drive. google. com/ file/ d/
> > > > > > > > > 14PRM7U0OopOOrJR197VlqvRX5SXNtmKj/ view?usp=sharing
> > > > > > > > > >>> (
> > > > > > > > > >>>
> > > > > > > > >
> > > > > >
> > > > >
> > https://drive.google.com/file/d/14PRM7U0OopOOrJR197VlqvRX5SXNtmKj/view?usp=sharing
> > > > > > > > > >>> )
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> Notes:
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> 1. KIP is updated with follower fetch protocol and
> > ready to
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> reviewed
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> 2. Satish to capture schema of internal metadata
> > topic in
> > > > > the
> > > > > > KIP
> > > > > > > > > >>>>>> 3. We will update the KIP with details of different
> > cases
> > > > > > > > > >>>>>> 4. Test plan will be captured in a doc and will add
> > to the
> > > > > KIP
> > > > > > > > > >>>>>> 5. Add a section "Limitations" to capture the
> > capabilities
> > > > > > that
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> will
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> be
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> introduced with this KIP and what will not be covered
> > in
> > > > > this
> > > > > > KIP.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> Please add to it I missed anything. Will produce a
> > formal
> > > > > > meeting
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> notes
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> from next meeting onwards.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> Thanks,
> > > > > > > > > >>>>>> Harsha
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> On Mon, Aug 24, 2020 at 9:42 PM, Ying Zheng < yingz@
> > uber.
> > > > > > com.
> > > > > > > > > invalid (
> > > > > > > > > >>>>>> yi...@uber.com.invalid ) > wrote:
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> We did some basic feature tests at Uber. The test
> > cases and
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> results are
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> shared in this google doc:
> > > > > > > > > >>>>>>> https:/ / docs. google. com/ spreadsheets/ d/ (
> > > > > > > > > >>>>>>> https://docs.google.com/spreadsheets/d/ )
> > > > > > > > > >>>>>>>
> > > > > 1XhNJqjzwXvMCcAOhEH0sSXU6RTvyoSf93DHF-YMfGLk/edit?usp=sharing
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> The performance test results were already shared in
> > the KIP
> > > > > > last
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> month.
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Mon, Aug 24, 2020 at 11:10 AM Harsha Ch < harsha.
> > ch@
> > > > > > gmail.
> > > > > > > > > com (
> > > > > > > > > >>>>>>> harsha...@gmail.com ) >
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> wrote:
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> "Understand commitments towards driving design &
> > > > > > implementation of
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> the
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> KIP
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> further and how it aligns with participant interests
> > in
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> contributing to
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> the
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> efforts (ex: in the context of Uber’s Q3/Q4
> > roadmap)." What
> > > > > > is that
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> about?
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Mon, Aug 24, 2020 at 11:05 AM Kowshik Prakasam <
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> kprakasam@ confluent. io ( kpraka...@confluent.io ) >
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> wrote:
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Hi Harsha,
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> The following google doc contains a proposal for
> > temporary
> > > > > > agenda
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> for
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> the
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> KIP-405 <
> > https://issues.apache.org/jira/browse/KIP-405> <
> > > > > > https:/ / issues. apache. org/ jira/ browse/ KIP-405
> > > > > > <https://issues.apache.org/jira/browse/KIP-405> (
> > > > > > > > > >>>>>>> https://issues.apache.org/jira/browse/KIP-405 ) >
> > sync
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> meeting
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> tomorrow:
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> https:/ / docs. google. com/ document/ d/ (
> > > > > > > > > >>>>>>> https://docs.google.com/document/d/ )
> > > > > > > > > >>>>>>> 1pqo8X5LU8TpwfC_iqSuVPezhfCfhGkbGN2TqiPA3LBU/edit
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> .
> > > > > > > > > >>>>>>> Please could you add it to the Google calendar
> > invite?
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Thank you.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Cheers,
> > > > > > > > > >>>>>>> Kowshik
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Thu, Aug 20, 2020 at 10:58 AM Harsha Ch < harsha.
> > ch@
> > > > > > gmail.
> > > > > > > > > com (
> > > > > > > > > >>>>>>> harsha...@gmail.com ) >
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> wrote:
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Hi All,
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Scheduled a meeting for Tuesday 9am - 10am. I can
> > record
> > > > > and
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> upload for
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> community to be able to follow the discussion.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Jun, please add the required folks on confluent side.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Harsha
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Thu, Aug 20, 2020 at 12:33 AM, Alexandre Dupriez <
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> alexandre.dupriez@
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> gmail. com ( http://gmail.com/ ) > wrote:
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Many thanks for your initiative.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> If you like, I am happy to attend at the time you
> > > > > suggested.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Many thanks,
> > > > > > > > > >>>>>>> Alexandre
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Le mer. 19 août 2020 à 22:00, Harsha Ch < harsha. ch@
> > > > > > gmail. com (
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> harsha.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> ch@ gmail. com ( c...@gmail.com ) ) > a écrit :
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Hi Jun,
> > > > > > > > > >>>>>>> Thanks. This will help a lot. Tuesday will work for
> > us.
> > > > > > > > > >>>>>>> -Harsha
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Wed, Aug 19, 2020 at 1:24 PM Jun Rao < jun@
> > confluent.
> > > > > > io (
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> jun@
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> confluent. io ( http://confluent.io/ ) ) > wrote:
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Hi, Satish, Ying, Harsha,
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Do you think it would be useful to have a regular
> > virtual
> > > > > > meeting
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> to
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> discuss this KIP? The goal of the meeting will be
> > sharing
> > > > > > > > > >>>>>>> design/development progress and discussing any open
> > issues
> > > > > to
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> accelerate
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> this KIP. If so, will every Tuesday (from next week)
> > > > > 9am-10am
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> PT
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> work for you? I can help set up a Zoom meeting,
> > invite
> > > > > > everyone who
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> might
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> be interested, have it recorded and shared, etc.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Thanks,
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Jun
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Tue, Aug 18, 2020 at 11:01 AM Satish Duggana <
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> satish. duggana@ gmail. com ( satish. duggana@
> > gmail. com
> > > > > (
> > > > > > > > > >>>>>>> satish.dugg...@gmail.com ) ) >
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> wrote:
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Hi Kowshik,
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Thanks for looking into the KIP and sending your
> > comments.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> 5001. Under the section "Follower fetch protocol in
> > > > > detail",
> > > > > > the
> > > > > > > > > >>>>>>> next-local-offset is the offset upto which the
> > segments are
> > > > > > copied
> > > > > > > > > >>>>>>>
> > > > > >
> >
> >

Reply via email to