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.