Hi, Abhijeet, Thanks for the reply.
The issue with recording the throttle time as a gauge is that it's transient. If the metric is not read immediately, the recorded value could be reset to 0. The admin won't realize that throttling has happened. For client quotas, the throttle time is tracked as the average throttle-time per user/client-id. This makes the metric less transient. Also, the configs use read/write whereas the metrics use fetch/copy. Could we make them consistent? Jun On Wed, Feb 28, 2024 at 6:49 AM Abhijeet Kumar <abhijeet.cse....@gmail.com> wrote: > Hi Jun, > > Clarified the meaning of the two metrics. Also updated the KIP. > > kafka.log.remote:type=RemoteLogManager, name=RemoteFetchThrottleTime -> The > duration of time required at a given moment to bring the observed fetch > rate within the allowed limit, by preventing further reads. > kafka.log.remote:type=RemoteLogManager, name=RemoteCopyThrottleTime -> The > duration of time required at a given moment to bring the observed remote > copy rate within the allowed limit, by preventing further copies. > > Regards. > > On Wed, Feb 28, 2024 at 12:28 AM Jun Rao <j...@confluent.io.invalid> wrote: > > > Hi, Abhijeet, > > > > Thanks for the explanation. Makes sense to me now. > > > > Just a minor comment. Could you document the exact meaning of the > following > > two metrics? For example, is the time accumulated? If so, is it from the > > start of the broker or over some window? > > > > kafka.log.remote:type=RemoteLogManager, name=RemoteFetchThrottleTime > > kafka.log.remote:type=RemoteLogManager, name=RemoteCopyThrottleTime > > > > Jun > > > > On Tue, Feb 27, 2024 at 1:39 AM Abhijeet Kumar < > abhijeet.cse....@gmail.com > > > > > wrote: > > > > > Hi Jun, > > > > > > The existing quota system for consumers is designed to throttle the > > > consumer if it exceeds the allowed fetch rate. > > > The additional quota we want to add works on the broker level. If the > > > broker-level remote read quota is being > > > exceeded, we prevent additional reads from the remote storage but do > not > > > prevent local reads for the consumer. > > > If the consumer has specified other partitions to read, which can be > > served > > > from local, it can continue to read those > > > partitions. To elaborate more, we make a check for quota exceeded if we > > > know a segment needs to be read from > > > remote. If the quota is exceeded, we simply skip the partition and move > > to > > > other segments in the fetch request. > > > This way consumers can continue to read the local data as long as they > > have > > > not exceeded the client-level quota. > > > > > > Also, when we choose the appropriate consumer-level quota, we would > > > typically look at what kind of local fetch > > > throughput is supported. If we were to reuse the same consumer quota, > we > > > should also consider the throughput > > > the remote storage supports. The throughput supported by remote may be > > > less/more than the throughput supported > > > by local (when using a cloud provider, it depends on the plan opted by > > the > > > user). The consumer quota has to be carefully > > > set considering both local and remote throughput. Instead, if we have a > > > separate quota, it makes things much simpler > > > for the user, since they already know what throughput their remote > > storage > > > supports. > > > > > > (Also, thanks for pointing out. I will update the KIP based on the > > > discussion) > > > > > > Regards, > > > Abhijeet. > > > > > > On Tue, Feb 27, 2024 at 2:49 AM Jun Rao <j...@confluent.io.invalid> > > wrote: > > > > > > > Hi, Abhijeet, > > > > > > > > Sorry for the late reply. It seems that you haven't updated the KIP > > based > > > > on the discussion? One more comment. > > > > > > > > 11. Currently, we already have a quota system for both the producers > > and > > > > consumers. I can understand why we need an additional > > > > remote.log.manager.write.quota.default quota. For example, when tier > > > > storage is enabled for the first time, there could be a lot of > segments > > > > that need to be written to the remote storage, even though there is > no > > > > increase in the produced data. However, I am not sure about an > > > > additional remote.log.manager.read.quota.default. The KIP says that > the > > > > reason is "This may happen when the majority of the consumers start > > > reading > > > > from the earliest offset of their respective Kafka topics.". However, > > > this > > > > can happen with or without tier storage and the current quota system > > for > > > > consumers is designed for solving this exact problem. Could you > explain > > > the > > > > usage of this additional quota? > > > > > > > > Thanks, > > > > > > > > Jun > > > > > > > > On Mon, Feb 12, 2024 at 11:08 AM Abhijeet Kumar < > > > > abhijeet.cse....@gmail.com> > > > > wrote: > > > > > > > > > Comments inline > > > > > > > > > > On Wed, Dec 6, 2023 at 1:12 AM Jun Rao <j...@confluent.io.invalid> > > > wrote: > > > > > > > > > > > Hi, Abhijeet, > > > > > > > > > > > > Thanks for the KIP. A few comments. > > > > > > > > > > > > 10. remote.log.manager.write.quota.default: > > > > > > 10.1 For other configs, we > > > > > > use replica.alter.log.dirs.io.max.bytes.per.second. To be > > consistent, > > > > > > perhaps this can be sth like > > > > > remote.log.manager.write.max.bytes.per.second. > > > > > > > > > > > > > > > > This makes sense, we can rename the following configs to be > > consistent. > > > > > > > > > > Remote.log.manager.write.quota.default -> > > > > > remote.log.manager.write.max.bytes.per.second > > > > > > > > > > Remote.log.manager.read.quota.default -> > > > > > remote.log.manager.read.max.bytes.per.second. > > > > > > > > > > > > > > > > > > > > > 10.2 Could we list the new metrics associated with the new quota. > > > > > > > > > > > > > > > > We will add the following metrics as mentioned in the other > response. > > > > > *RemoteFetchThrottleTime* - The amount of time needed to bring the > > > > observed > > > > > remote fetch rate within the read quota > > > > > *RemoteCopyThrottleTime *- The amount of time needed to bring the > > > > observed > > > > > remote copy rate with the copy quota. > > > > > > > > > > 10.3 Is this dynamically configurable? If so, could we document the > > > > impact > > > > > > to tools like kafka-configs.sh and AdminClient? > > > > > > > > > > > > > > > > Yes, the quotas are dynamically configurable. We will add them as > > > Dynamic > > > > > Broker Configs. Users will be able to change > > > > > the following configs using either kafka-configs.sh or AdminClient > by > > > > > specifying the config name and new value. For eg. > > > > > > > > > > Using kafka-configs.sh > > > > > > > > > > bin/kafka-configs.sh --bootstrap-server <bootstrap-server> > > > --entity-type > > > > > brokers --entity-default --alter --add-config > > > > > remote.log.manager.write.max.bytes.per.second=52428800 > > > > > > > > > > Using AdminClient > > > > > > > > > > ConfigEntry configEntry = new > > > > > ConfigEntry("remote.log.manager.write.max.bytes.per.second", > > > "5242800"); > > > > > AlterConfigOp alterConfigOp = new AlterConfigOp(configEntry, > > > > > AlterConfigOp.OpType.SET); > > > > > List<AlterConfigOp> alterConfigOps = > > > > > Collections.singletonList(alterConfigOp); > > > > > > > > > > ConfigResource resource = new > > > ConfigResource(ConfigResource.Type.BROKER, > > > > > ""); > > > > > Map<ConfigResource, Collection<AlterConfigOp>> updateConfig = > > > > > ImmutableMap.of(resource, alterConfigOps); > > > > > adminClient.incrementalAlterConfigs(updateConfig); > > > > > > > > > > > > > > > > > > > > > > Jun > > > > > > > > > > > > On Tue, Nov 28, 2023 at 2:19 AM Luke Chen <show...@gmail.com> > > wrote: > > > > > > > > > > > > > Hi Abhijeet, > > > > > > > > > > > > > > Thanks for the KIP! > > > > > > > This is an important feature for tiered storage. > > > > > > > > > > > > > > Some comments: > > > > > > > 1. Will we introduce new metrics for this tiered storage > quotas? > > > > > > > This is important because the admin can know the throttling > > status > > > by > > > > > > > checking the metrics while the remote write/read are slow, like > > the > > > > > rate > > > > > > of > > > > > > > uploading/reading byte rate, the throttled time for > > upload/read... > > > > etc. > > > > > > > > > > > > > > 2. Could you give some examples for the throttling algorithm in > > the > > > > KIP > > > > > > to > > > > > > > explain it? That will make it much clearer. > > > > > > > > > > > > > > 3. To solve this problem, we can break down the RLMTask into > two > > > > > smaller > > > > > > > tasks - one for segment upload and the other for handling > expired > > > > > > segments. > > > > > > > How do we handle the situation when a segment is still waiting > > for > > > > > > > offloading while this segment is expired and eligible to be > > > deleted? > > > > > > > Maybe it'll be easier to not block the RLMTask when quota > > exceeded, > > > > and > > > > > > > just check it each time the RLMTask runs? > > > > > > > > > > > > > > Thank you. > > > > > > > Luke > > > > > > > > > > > > > > On Wed, Nov 22, 2023 at 6:27 PM Abhijeet Kumar < > > > > > > abhijeet.cse....@gmail.com > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > Hi All, > > > > > > > > > > > > > > > > I have created KIP-956 for defining read and write quota for > > > tiered > > > > > > > > storage. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-956+Tiered+Storage+Quotas > > > > > > > > > > > > > > > > Feedback and suggestions are welcome. > > > > > > > > > > > > > > > > Regards, > > > > > > > > Abhijeet. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > Abhijeet. > > > > > > > > > > > > > > > > > > -- > > > Abhijeet. > > > > > > > > -- > Abhijeet. >