chia7712 commented on code in PR #17609:
URL: https://github.com/apache/kafka/pull/17609#discussion_r1821097842


##########
core/src/main/java/kafka/server/ReplicationQuotaManager.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.server;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+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.apache.kafka.server.config.ReplicationQuotaManagerConfig;
+import org.apache.kafka.server.quota.QuotaType;
+import org.apache.kafka.server.quota.SensorAccess;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class ReplicationQuotaManager implements ReplicaQuota {
+    public static final List<Integer> ALL_REPLICAS = List.of(-1);
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(ReplicationQuotaManager.class);
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final ConcurrentHashMap<String, List<Integer>> throttledPartitions 
= new ConcurrentHashMap<>();
+    private final ReplicationQuotaManagerConfig config;
+    private final Metrics metrics;
+    private final QuotaType replicationType;
+    private final Time time;
+    private final SensorAccess sensorAccess;
+    private final MetricName rateMetricName;
+    private Quota quota;
+
+    public ReplicationQuotaManager(ReplicationQuotaManagerConfig config, 
Metrics metrics, QuotaType replicationType, Time time) {
+        this.config = config;
+        this.metrics = metrics;
+        this.replicationType = replicationType;
+        this.time = time;
+        this.sensorAccess = new SensorAccess(lock, metrics);
+        this.rateMetricName = metrics.metricName("byte-rate", 
replicationType.toString(), "Tracking byte-rate for " + replicationType);
+    }
+
+    /**
+     * Update the quota
+     */
+    public void updateQuota(Quota quota) {
+        lock.writeLock().lock();
+        try {
+            this.quota = quota;
+            KafkaMetric metric = metrics.metrics().get(rateMetricName);
+            if (metric != null) {
+                metric.config(getQuotaMetricConfig(quota));
+            }
+        } finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Check if the quota is currently exceeded
+     */
+    @Override
+    public boolean isQuotaExceeded() {
+        try {

Review Comment:
   ```java
           try {
               sensor().checkQuotas();
               return false;
           } catch (QuotaViolationException qve) {
               LOGGER.trace("{}: Quota violated for sensor ({}), metric: ({}), 
metric-value: ({}), bound: ({})",
                   replicationType, sensor().name(), qve.metric().metricName(), 
qve.value(), qve.bound());
               return true;
           }
   ```



##########
core/src/main/java/kafka/server/ClientRequestQuotaManager.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.server;
+
+import kafka.network.RequestChannel;
+
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.QuotaViolationException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Rate;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.config.ClientQuotaManagerConfig;
+import org.apache.kafka.server.quota.ClientQuotaCallback;
+import org.apache.kafka.server.quota.QuotaType;
+import org.apache.kafka.server.quota.QuotaUtils;
+
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import scala.jdk.javaapi.CollectionConverters;
+import scala.jdk.javaapi.OptionConverters;
+import scala.runtime.BoxedUnit;
+
+@SuppressWarnings("this-escape")
+public class ClientRequestQuotaManager extends ClientQuotaManager {
+    // Since exemptSensor is for all clients and has a constant name, we do 
not expire exemptSensor and only
+    // create once.
+    static final double NANOS_TO_PERCENTAGE_PER_SECOND = 100.0 / 
TimeUnit.SECONDS.toNanos(1);
+    private static final long 
DEFAULT_INACTIVE_EXEMPT_SENSOR_EXPIRATION_TIME_SECONDS = Long.MAX_VALUE;
+    private static final String EXEMPT_SENSOR_NAME = "exempt-" + 
QuotaType.REQUEST;
+
+    private final long maxThrottleTimeMs;
+    private final Metrics metrics;
+    private final MetricName exemptMetricName;
+    // Visible for testing
+    private final Sensor exemptSensor;
+
+    public ClientRequestQuotaManager(ClientQuotaManagerConfig config, Metrics 
metrics, Time time, String threadNamePrefix, Optional<ClientQuotaCallback> 
quotaCallback) {
+        super(config, metrics, QuotaType.REQUEST, time, threadNamePrefix, 
OptionConverters.toScala(quotaCallback));
+        this.maxThrottleTimeMs = 
TimeUnit.SECONDS.toMillis(config.quotaWindowSizeSeconds);
+        this.metrics = metrics;
+        this.exemptMetricName = metrics.metricName("exempt-request-time", 
QuotaType.REQUEST.toString(), "Tracking exempt-request-time utilization 
percentage");
+        exemptSensor = getOrCreateSensor(EXEMPT_SENSOR_NAME, 
DEFAULT_INACTIVE_EXEMPT_SENSOR_EXPIRATION_TIME_SECONDS, sensor -> 
sensor.add(exemptMetricName, new Rate()));
+    }
+
+    public Sensor exemptSensor() {
+        return exemptSensor;
+    }
+
+    private void recordExempt(double value) {
+        exemptSensor.record(value);
+    }
+
+    /**
+     * Records that a user/clientId changed request processing time being 
throttled. If quota has been violated, return
+     * throttle time in milliseconds. Throttle time calculation may be 
overridden by sub-classes.
+     * @param request client request
+     * @return Number of milliseconds to throttle in case of quota violation. 
Zero otherwise
+     */
+    public int maybeRecordAndGetThrottleTimeMs(RequestChannel.Request request, 
long timeMs) {
+        if (quotasEnabled()) {
+            request.setRecordNetworkThreadTimeCallback(timeNanos -> {

Review Comment:
   Could you please change the input type of 
`setRecordNetworkThreadTimeCallback` from `Long => Unit` to 
`java.util.function.Consumer`? Using the scala type in java is a bit weird



##########
core/src/main/java/kafka/server/QuotaFactory.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.server;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.server.config.ClientQuotaManagerConfig;
+import org.apache.kafka.server.config.QuotaConfig;
+import org.apache.kafka.server.config.ReplicationQuotaManagerConfig;
+import org.apache.kafka.server.quota.ClientQuotaCallback;
+import org.apache.kafka.server.quota.QuotaType;
+
+import java.util.Optional;
+
+import scala.Option;
+
+public class QuotaFactory {
+
+    public static final UnboundedQuota UNBOUNDED_QUOTA = new UnboundedQuota();
+
+    public static class UnboundedQuota implements ReplicaQuota {

Review Comment:
   ```java
       public static final ReplicaQuota UNBOUNDED_QUOTA = new ReplicaQuota() {
           @Override
           public boolean isThrottled(TopicPartition topicPartition) {
               return false;
           }
   
           @Override
           public boolean isQuotaExceeded() {
               return false;
           }
   
           @Override
           public void record(long value) {
               // No-op
           }
       };
   ```
   WDYT?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to