-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-15265: Add Remote Log Manager quota manager #15625
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
0f20308
7ec240e
21f5363
a2048f2
a4f3d63
d57b8c3
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I have a question, in fact, what this PR does is to provide a standard configuration and generate corresponding rate limiters and related monitoring indicators, right? Then it needs to be used in the corresponding RemoteStorageManager, correct?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, the integration of the quota manager will come in the follow-up PRs. I have mentioned it in the description of the PR. |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,117 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package kafka.log.remote.quota; | ||
|
|
||
| import kafka.server.QuotaType; | ||
| import kafka.server.SensorAccess; | ||
| import org.apache.kafka.common.MetricName; | ||
| import org.apache.kafka.common.metrics.KafkaMetric; | ||
| import org.apache.kafka.common.metrics.MetricConfig; | ||
| import org.apache.kafka.common.metrics.Metrics; | ||
| import org.apache.kafka.common.metrics.Quota; | ||
| import org.apache.kafka.common.metrics.QuotaViolationException; | ||
| import org.apache.kafka.common.metrics.Sensor; | ||
| import org.apache.kafka.common.metrics.stats.SimpleRate; | ||
| import org.apache.kafka.common.utils.Time; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
| import scala.runtime.BoxedUnit; | ||
|
|
||
| import java.util.Collections; | ||
| import java.util.Map; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
|
|
||
| public class RLMQuotaManager { | ||
|
abhijeetk88 marked this conversation as resolved.
|
||
| private static final Logger LOGGER = LoggerFactory.getLogger(RLMQuotaManager.class); | ||
|
|
||
| private final RLMQuotaManagerConfig config; | ||
| private final Metrics metrics; | ||
| private final QuotaType quotaType; | ||
| private final String description; | ||
| private final Time time; | ||
|
|
||
| private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); | ||
| private final SensorAccess sensorAccess; | ||
| private Quota quota; | ||
|
|
||
| public RLMQuotaManager(RLMQuotaManagerConfig config, Metrics metrics, QuotaType quotaType, String description, Time time) { | ||
| this.config = config; | ||
| this.metrics = metrics; | ||
| this.quotaType = quotaType; | ||
| this.description = description; | ||
| this.time = time; | ||
|
|
||
| this.quota = new Quota(config.quotaBytesPerSecond(), true); | ||
| this.sensorAccess = new SensorAccess(lock, metrics); | ||
| } | ||
|
|
||
| public void updateQuota(Quota newQuota) { | ||
| lock.writeLock().lock(); | ||
| try { | ||
| this.quota = newQuota; | ||
|
|
||
| Map<MetricName, KafkaMetric> allMetrics = metrics.metrics(); | ||
| MetricName quotaMetricName = metricName(); | ||
| KafkaMetric metric = allMetrics.get(quotaMetricName); | ||
| if (metric != null) { | ||
| LOGGER.info("Sensor for quota-id {} already exists. Setting quota to {} in MetricConfig", quotaMetricName, newQuota); | ||
| metric.config(getQuotaMetricConfig(newQuota)); | ||
| } | ||
| } finally { | ||
| lock.writeLock().unlock(); | ||
| } | ||
| } | ||
|
|
||
| public boolean isQuotaExceeded() { | ||
| Sensor sensorInstance = sensor(); | ||
| try { | ||
| sensorInstance.checkQuotas(); | ||
| } catch (QuotaViolationException qve) { | ||
| LOGGER.debug("Quota violated for sensor ({}), metric: ({}), metric-value: ({}), bound: ({})", | ||
|
abhijeetk88 marked this conversation as resolved.
|
||
| sensorInstance.name(), qve.metric().metricName(), qve.value(), qve.bound()); | ||
| return true; | ||
| } | ||
| return false; | ||
| } | ||
|
|
||
| public void record(double value) { | ||
| sensor().record(value, time.milliseconds(), false); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we turn the quota checking off during
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In KIP-956, we do not utilize the throttle time provided by the quota manager to regulate fetches and copies. For fetch operations, we initially verify quota availability before initiating the retrieval of remote data. If the quota is unavailable, our priority is to serve partitions requiring local data, rather than throttling the client. Therefore, we focus on fulfilling data requests for other partitions in the queue, eliminating the need for throttle time in fetch operations. Similarly, when a RLM Task attempts to copy a segment, it first checks if the write quota is available. If the quota is not available, the thread waits until the quota becomes available. As a result, we do not require throttle time for copies either. |
||
| } | ||
|
|
||
| private MetricConfig getQuotaMetricConfig(Quota quota) { | ||
| return new MetricConfig() | ||
| .timeWindow(config.quotaWindowSizeSeconds(), TimeUnit.SECONDS) | ||
| .samples(config.numQuotaSamples()) | ||
| .quota(quota); | ||
| } | ||
|
|
||
| private MetricName metricName() { | ||
| return metrics.metricName("byte-rate", quotaType.toString(), description, Collections.emptyMap()); | ||
| } | ||
|
|
||
| private Sensor sensor() { | ||
| return sensorAccess.getOrCreate( | ||
| quotaType.toString(), | ||
| RLMQuotaManagerConfig.INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS, | ||
| sensor -> { | ||
| sensor.add(metricName(), new SimpleRate(), getQuotaMetricConfig(quota)); | ||
| return BoxedUnit.UNIT; | ||
| } | ||
| ); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,59 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package kafka.log.remote.quota; | ||
|
|
||
| public class RLMQuotaManagerConfig { | ||
|
abhijeetk88 marked this conversation as resolved.
|
||
| public static final int INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS = 3600; | ||
|
|
||
| private final long quotaBytesPerSecond; | ||
| private final int numQuotaSamples; | ||
| private final int quotaWindowSizeSeconds; | ||
|
|
||
| /** | ||
| * Configuration settings for quota management | ||
| * | ||
| * @param quotaBytesPerSecond The quota in bytes per second | ||
| * @param numQuotaSamples The number of samples to retain in memory | ||
| * @param quotaWindowSizeSeconds The time span of each sample | ||
| */ | ||
| public RLMQuotaManagerConfig(long quotaBytesPerSecond, int numQuotaSamples, int quotaWindowSizeSeconds) { | ||
|
abhijeetk88 marked this conversation as resolved.
|
||
| this.quotaBytesPerSecond = quotaBytesPerSecond; | ||
| this.numQuotaSamples = numQuotaSamples; | ||
| this.quotaWindowSizeSeconds = quotaWindowSizeSeconds; | ||
| } | ||
|
|
||
| public long quotaBytesPerSecond() { | ||
| return quotaBytesPerSecond; | ||
| } | ||
|
|
||
| public int numQuotaSamples() { | ||
| return numQuotaSamples; | ||
| } | ||
|
|
||
| public int quotaWindowSizeSeconds() { | ||
| return quotaWindowSizeSeconds; | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "RLMQuotaManagerConfig{" + | ||
| "quotaBytesPerSecond=" + quotaBytesPerSecond + | ||
| ", numQuotaSamples=" + numQuotaSamples + | ||
| ", quotaWindowSizeSeconds=" + quotaWindowSizeSeconds + | ||
| '}'; | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.