Thank you for spotting that Luke. I have fixed the snippet now.
*Satish*, I am waiting for your review on this one since you provided some
comments earlier in this discussion. Please check the KIP once again when
you get a chance and vote at
https://lists.apache.org/thread/soz00990gvzodv7oyqj4ysvk
Thanks Divij.
I was confusing with the metric tags used by clients that are based on
topic and partition. Ideally partition label could be at a DEBUG recording
level, but that's outside the scope of this KIP.
Looks good to me, thanks again!
Jorge.
On Wed, 12 Jul 2023 at 15:55, Divij Vaidya wro
Hi Divij,
One minor comment:
remoteLogSize takes 2 parameters, but in the code snippet, you only provide
1 parameter.
Otherwise, LGTM
Thank you.
Luke
On Wed, Jul 12, 2023 at 8:56 PM Divij Vaidya
wrote:
> Jorge,
> About API name: Good point. I have changed it to remoteLogSize instead of
> getR
Jorge,
About API name: Good point. I have changed it to remoteLogSize instead of
getRemoteLogSize
About partition tag in the metric: We don't use partition tag across any of
the RemoteStorage metrics and I would like to keep this metric aligned with
the rest. I will change the metric though to typ
Thanks Divij for taking the feedback and updating the motivation
section in the KIP.
One more comment on Alternative solution-3, The con is not valid as
that will not affect the broker restart times as discussed in the
earlier email in this thread. You may want to update that.
~Satish.
On Sun, 2
Thanks Divij, this KIP is a super useful improvement to Tiered Storage.
I have a couple of minor comments to the KIP, otherwise I'm +1 on this
proposal:
1. APIs haven't used getter naming convention on TS as far as I can see
(e.g `RLMM#remoteLogSegmentMetadata()`). We could rename the proposed
me
Thank you folks for reviewing this KIP.
Satish, I have modified the motivation to make it more clear. Now it says,
"Since the main feature of tiered storage is storing a large amount of
data, we expect num_remote_segments to be large. A frequent linear scan
(i.e. listing all segment metadata) coul
Hi Divij,
Thanks for the explanation. LGTM.
--
Kamal
On Sat, Jul 1, 2023 at 7:28 AM Satish Duggana
wrote:
> Hi Divij,
> I am fine with having an API to compute the size as I mentioned in my
> earlier reply in this mail thread. But I have the below comment for
> the motivation for this KIP.
>
>
Hi Divij,
I am fine with having an API to compute the size as I mentioned in my
earlier reply in this mail thread. But I have the below comment for
the motivation for this KIP.
As you discussed offline, the main issue here is listing calls for
remote log segment metadata is slower because of the s
Hi, Divij,
Sorry for the late reply.
Given your explanation, the new API sounds reasonable to me. Is that enough
to build the external metadata layer for the remote segments or do you need
some additional API changes?
Thanks,
Jun
On Fri, Jun 9, 2023 at 7:08 AM Divij Vaidya wrote:
> Thank you
Thank you for looking into this Kamal.
You are right in saying that a cold start (i.e. leadership failover or
broker startup) does not impact the broker startup duration. But it does
have the following impact:
1. It leads to a burst of full-scan requests to RLMM in case multiple
leadership failove
Hi Divij,
Thanks for the KIP! Sorry for the late reply.
Can you explain the rejected alternative-3?
Store the cumulative size of remote tier log in-memory at RemoteLogManager
"*Cons*: Every time a broker starts-up, it will scan through all the
segments in the remote tier to initialise the in-memo
Satish / Jun
Do you have any thoughts on this?
--
Divij Vaidya
On Tue, Feb 14, 2023 at 4:15 PM Divij Vaidya
wrote:
> Hey Jun
>
> It has been a while since this KIP got some attention. While we wait for
> Satish to chime in here, perhaps I can answer your question.
>
> > Could you explain how
Hey Jun
It has been a while since this KIP got some attention. While we wait for
Satish to chime in here, perhaps I can answer your question.
> Could you explain how you exposed the log size in your KIP-405
implementation?
The APIs available in RLMM as per KIP405
are, addRemoteLogSegmentMetadata
Hi, Divij,
Thanks for the explanation.
Good question.
Hi, Satish,
Could you explain how you exposed the log size in your KIP-405
implementation?
Thanks,
Jun
On Tue, Dec 20, 2022 at 4:59 AM Divij Vaidya
wrote:
> Hey Jun
>
> Yes, it is possible to maintain the log size in the cache (see reje
Hey Jun
Yes, it is possible to maintain the log size in the cache (see rejected
alternative#3 in the KIP) but I did not understand how it is possible to
retrieve it without the new API. The log size could be calculated on
startup by scanning through the segments (though I would disagree that this
Hi, Divij,
Thanks for the explanation.
If there is in-memory cache, could we maintain the log size in the cache
with the existing API? For example, a replica could make a
listRemoteLogSegments(TopicIdPartition topicIdPartition) call on startup to
get the remote segment size before the current lea
Thanks Divij for the KIP.
I wanted to clarify on a couple of points mentioned in your earlier reply.
1. RSM fetch interface already exposes a streaming interface and it also
does not mandate reading the complete segment before serving the fetch
request. Let me know if we are missing anything here
> Is the new method enough for doing size-based retention?
Yes. You are right in assuming that this API only provides the Remote
storage size (for current epoch chain). We would use this API for size
based retention along with a value of localOnlyLogSegmentSize which is
computed as Log.sizeInBytes
Hi, Divij,
Thanks for the reply.
Is the new method enough for doing size-based retention? It gives the total
size of the remote segments, but it seems that we still don't know the
exact total size for a log since there could be overlapping segments
between the remote and the local segments.
You
The method is needed for RLMM implementations which fetch the information
over the network and not for the disk based implementations (such as the
default topic based RLMM).
I would argue that adding this API makes the interface more generic than
what it is today. This is because, with the current
Hi, Divij,
Thanks for the reply.
Point#2. My high level question is that is the new method needed for every
implementation of remote storage or just for a specific implementation. The
issues that you pointed out exist for the default implementation of RLMM as
well and so far, the default implemen
Thank you Jun and Alex for your comments.
Point#1: You are right Jun. As Alex mentioned, the "derived metadata" can
increase the size of cached metadata by a factor of 10 but it should be ok
to cache just the actual metadata. My point about size being a limitation
for using cache is not valid anym
Hi Divij,
Thanks for the KIP. Please find some comments based on what I read on
this thread so far - apologies for the repeats and the late reply.
If I understand correctly, one of the main elements of discussion is
about caching in Kafka versus delegation of providing the remote size
of a topic-
Hi, Divij,
Thanks for the reply.
Point #1. Is the average remote segment metadata really 1KB? What's listed
in the public interface is probably well below 100 bytes.
Point #2. I guess you are assuming that each broker only caches the remote
segment metadata in memory. An alternative approach is
Hi Jun
There are three points that I would like to present here:
1. We would require a large cache size to efficiently cache all segment
metadata.
2. Linear scan of all metadata at broker startup to populate the cache will
be slow and will impact the archival process.
3. There is no other use cas
Hi, Divij,
Thanks for the reply.
It's true that the data in Kafka could be kept longer with KIP-405. How
much data do you envision to have per broker? For 100TB data per broker,
with 1GB segment and segment metadata of 100 bytes, it requires
100TB/1GB*100 = 10MB, which should fit in memory.
Remo
*Jun,*
*"the default implementation of RLMM does local caching, right?"*
Yes, Jun. The default implementation of RLMM does indeed cache the segment
metadata today, hence, it won't work for use cases when the number of
segments in remote storage is large enough to exceed the size of cache. As
part
Hi Divij,
One more question about the metric:
I think the metric will be updated when
(1) each time we run the log retention check (that is,
log.retention.check.interval.ms)
(2) When user explicitly call getRemoteLogSize
Is that correct?
Maybe we should add a note in metric description, otherwise
Hi, Divij,
Thanks for the explanation.
1. Hmm, the default implementation of RLMM does local caching, right?
Currently, we also cache all segment metadata in the brokers without
KIP-405. Do you see a need to change that?
2,3,4: Yes, your explanation makes sense. However,
currently, RemoteLogMeta
Hey Jun
Thank you for your comments.
*1. "RLMM implementor could ensure that listRemoteLogSegments() is fast"*
This would be ideal but pragmatically, it is difficult to ensure that
listRemoteLogSegments() is fast. This is because of the possibility of a
large number of segments (much larger than
Hi, Divij,
Thanks for the KIP. Sorry for the late reply.
The motivation of the KIP is to improve the efficiency of size based
retention. I am not sure the proposed changes are enough. For example, if
the size exceeds the retention size, we need to determine the subset of
segments to delete to bri
Hey folks
Does anyone else have any thoughts on this before I propose this for a
vote?
--
Divij Vaidya
On Mon, Sep 5, 2022 at 12:57 PM Satish Duggana
wrote:
> Thanks for the KIP Divij!
>
> This is a nice improvement to avoid recalculation of size. Customized RLMMs
> can implement the best po
Thanks for the KIP Divij!
This is a nice improvement to avoid recalculation of size. Customized RLMMs
can implement the best possible approach by caching or maintaining the size
in an efficient way. But this is not a big concern for the default topic
based RLMM as mentioned in the KIP.
~Satish.
Thank you for your review Luke.
> Reg: is that would the new `RemoteLogSizeBytes` metric be a performance
overhead? Although we move the calculation to a seperate API, we still
can't assume users will implement a light-weight method, right?
This metric would be logged using the information that i
Hi Divij,
Thanks for the KIP!
I think it makes sense to delegate the responsibility of calculation to the
specific RemoteLogMetadataManager implementation.
But one thing I'm not quite sure, is that would the new
`RemoteLogSizeBytes` metric be a performance overhead?
Although we move the calculati
https://cwiki.apache.org/confluence/display/KAFKA/KIP-852%3A+Optimize+calculation+of+size+for+log+in+remote+tier
Hey folks
Please take a look at this KIP which proposes an extension to KIP-405. This
is my first KIP with Apache Kafka community so any feedback would be highly
appreciated.
Cheers!
37 matches
Mail list logo