kamalcph commented on code in PR #15625:
URL: https://github.com/apache/kafka/pull/15625#discussion_r1577552070


##########
core/src/main/java/kafka/log/remote/quota/RLMQuotaManagerConfig.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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 {

Review Comment:
   do we need to add `toString()`? 



##########
core/src/test/java/kafka/log/remote/quota/RLMQuotaManagerTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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 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.utils.MockTime;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class RLMQuotaManagerTest {
+    private final MockTime time = new MockTime();
+    private final Metrics metrics = new Metrics(new MetricConfig(), 
Collections.emptyList(), time);
+    private static final QuotaType QUOTA_TYPE = QuotaType.RLMFetch$.MODULE$;
+    private static final String DESCRIPTION = "Tracking byte rate";
+
+    @Test
+    public void testQuotaExceeded() {
+        RLMQuotaManager quotaManager = new RLMQuotaManager(
+            new RLMQuotaManagerConfig(50, 11, 1), metrics, QUOTA_TYPE, 
DESCRIPTION, time);
+
+        assertFalse(quotaManager.isQuotaExceeded());
+        quotaManager.record(500);
+        // Move clock by 1 sec, quota is violated
+        moveClock(1);
+        assertTrue(quotaManager.isQuotaExceeded());
+
+        // Move clock by another 8 secs, quota is still violated for the window
+        moveClock(8);
+        assertTrue(quotaManager.isQuotaExceeded());
+
+        // Move clock by 1 sec, quota is no more violated
+        moveClock(1);
+        assertFalse(quotaManager.isQuotaExceeded());
+    }
+
+    @Test
+    public void testQuotaUpdate() {
+        RLMQuotaManager quotaManager = new RLMQuotaManager(
+            new RLMQuotaManagerConfig(50, 11, 1), metrics, QUOTA_TYPE, 
DESCRIPTION, time);
+
+        assertFalse(quotaManager.isQuotaExceeded());
+        quotaManager.record(51);
+        assertTrue(quotaManager.isQuotaExceeded());
+
+        Map<MetricName, KafkaMetric> fetchQuotaMetrics = 
metrics.metrics().entrySet().stream()
+            .filter(entry -> entry.getKey().name().equals("byte-rate") && 
entry.getKey().group().equals(QUOTA_TYPE.toString()))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        Map<MetricName, KafkaMetric> nonQuotaMetrics = 
metrics.metrics().entrySet().stream()
+            .filter(entry -> !entry.getKey().name().equals("byte-rate") || 
!entry.getKey().group().equals(QUOTA_TYPE.toString()))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        assertEquals(1, fetchQuotaMetrics.size());
+        assertFalse(nonQuotaMetrics.isEmpty());
+
+        Map<MetricName, MetricConfig> configForQuotaMetricsBeforeUpdate = 
extractMetricConfig(fetchQuotaMetrics);
+        Map<MetricName, MetricConfig> configForNonQuotaMetricsBeforeUpdate = 
extractMetricConfig(nonQuotaMetrics);
+
+        // Update quota to 60, quota is no more violated
+        Quota quota60Bytes = new Quota(60, true);
+        quotaManager.updateQuota(quota60Bytes);
+        assertFalse(quotaManager.isQuotaExceeded());
+
+        // Verify quota metrics were updated
+        Map<MetricName, MetricConfig> configForQuotaMetricsAfterFirstUpdate = 
extractMetricConfig(fetchQuotaMetrics);
+        assertNotEquals(configForQuotaMetricsBeforeUpdate, 
configForQuotaMetricsAfterFirstUpdate);
+        fetchQuotaMetrics.values().forEach(metric -> 
assertEquals(metric.config().quota(), quota60Bytes));

Review Comment:
   consider swapping: 
   
   assertEquals(actual, expected) to assertEquals(expected, actual)



##########
core/src/test/java/kafka/log/remote/quota/RLMQuotaManagerTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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 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.utils.MockTime;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class RLMQuotaManagerTest {
+    private final MockTime time = new MockTime();
+    private final Metrics metrics = new Metrics(new MetricConfig(), 
Collections.emptyList(), time);
+    private static final QuotaType QUOTA_TYPE = QuotaType.RLMFetch$.MODULE$;
+    private static final String DESCRIPTION = "Tracking byte rate";
+
+    @Test
+    public void testQuotaExceeded() {
+        RLMQuotaManager quotaManager = new RLMQuotaManager(
+            new RLMQuotaManagerConfig(50, 11, 1), metrics, QUOTA_TYPE, 
DESCRIPTION, time);
+
+        assertFalse(quotaManager.isQuotaExceeded());
+        quotaManager.record(500);
+        // Move clock by 1 sec, quota is violated
+        moveClock(1);
+        assertTrue(quotaManager.isQuotaExceeded());
+
+        // Move clock by another 8 secs, quota is still violated for the window
+        moveClock(8);
+        assertTrue(quotaManager.isQuotaExceeded());
+
+        // Move clock by 1 sec, quota is no more violated
+        moveClock(1);
+        assertFalse(quotaManager.isQuotaExceeded());
+    }
+
+    @Test
+    public void testQuotaUpdate() {
+        RLMQuotaManager quotaManager = new RLMQuotaManager(
+            new RLMQuotaManagerConfig(50, 11, 1), metrics, QUOTA_TYPE, 
DESCRIPTION, time);
+
+        assertFalse(quotaManager.isQuotaExceeded());
+        quotaManager.record(51);
+        assertTrue(quotaManager.isQuotaExceeded());
+
+        Map<MetricName, KafkaMetric> fetchQuotaMetrics = 
metrics.metrics().entrySet().stream()
+            .filter(entry -> entry.getKey().name().equals("byte-rate") && 
entry.getKey().group().equals(QUOTA_TYPE.toString()))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        Map<MetricName, KafkaMetric> nonQuotaMetrics = 
metrics.metrics().entrySet().stream()
+            .filter(entry -> !entry.getKey().name().equals("byte-rate") || 
!entry.getKey().group().equals(QUOTA_TYPE.toString()))
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        assertEquals(1, fetchQuotaMetrics.size());
+        assertFalse(nonQuotaMetrics.isEmpty());
+
+        Map<MetricName, MetricConfig> configForQuotaMetricsBeforeUpdate = 
extractMetricConfig(fetchQuotaMetrics);
+        Map<MetricName, MetricConfig> configForNonQuotaMetricsBeforeUpdate = 
extractMetricConfig(nonQuotaMetrics);
+
+        // Update quota to 60, quota is no more violated
+        Quota quota60Bytes = new Quota(60, true);
+        quotaManager.updateQuota(quota60Bytes);
+        assertFalse(quotaManager.isQuotaExceeded());
+
+        // Verify quota metrics were updated
+        Map<MetricName, MetricConfig> configForQuotaMetricsAfterFirstUpdate = 
extractMetricConfig(fetchQuotaMetrics);
+        assertNotEquals(configForQuotaMetricsBeforeUpdate, 
configForQuotaMetricsAfterFirstUpdate);
+        fetchQuotaMetrics.values().forEach(metric -> 
assertEquals(metric.config().quota(), quota60Bytes));
+        // Verify non quota metrics are unchanged
+        assertEquals(configForNonQuotaMetricsBeforeUpdate, 
extractMetricConfig(nonQuotaMetrics));
+
+        // Update quota to 40, quota is violated again
+        Quota quota40Bytes = new Quota(40, true);
+        quotaManager.updateQuota(quota40Bytes);
+        assertTrue(quotaManager.isQuotaExceeded());
+
+        // Verify quota metrics were updated
+        assertNotEquals(configForQuotaMetricsAfterFirstUpdate, 
extractMetricConfig(fetchQuotaMetrics));
+        fetchQuotaMetrics.values().forEach(metric -> 
assertEquals(metric.config().quota(), quota40Bytes));

Review Comment:
   consider swapping:
   
   assertEquals(actual, expected) to assertEquals(expected, actual)



##########
core/src/main/java/kafka/log/remote/quota/RLMQuotaManager.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 {
+    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.getQuotaBytesPerSecond(), 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();
+            allMetrics.forEach((metricName, metric) -> {
+                if (metricName.name().equals(quotaMetricName.name()) && 
metricName.group().equals(quotaMetricName.group())) {
+                    Map<String, String> metricTags = metricName.tags();
+                    LOGGER.info("Sensor for quota-id {} already exists. 
Setting quota to {} in MetricConfig", metricTags, newQuota);
+                    metric.config(getQuotaMetricConfig(newQuota));

Review Comment:
   ```suggestion
       KafkaMetric metric = allMetrics.get(quotaMetricName);
       if (metric != null) {
           Map<String, String> metricTags = quotaMetricName.tags();
           LOGGER.info("Sensor for quota-id {} already exists. Setting quota to 
{} in MetricConfig", metricTags, newQuota);
           metric.config(getQuotaMetricConfig(newQuota));
       }
   ```
   why are we not accessing the map directly similar to other implementations? 
(ie) why `metricTags` is excluded?



##########
core/src/main/java/kafka/log/remote/quota/RLMQuotaManager.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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 {
+    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.getQuotaBytesPerSecond(), 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();
+            allMetrics.forEach((metricName, metric) -> {
+                if (metricName.name().equals(quotaMetricName.name()) && 
metricName.group().equals(quotaMetricName.group())) {
+                    Map<String, String> metricTags = metricName.tags();
+                    LOGGER.info("Sensor for quota-id {} already exists. 
Setting quota to {} in MetricConfig", metricTags, 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: ({})",

Review Comment:
   metricName contains the quota-type. Do we need to log the quota-type similar 
to other quota-manager implementations?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to