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


##########
core/src/test/java/kafka/test/api/CustomQuotaCallbackTest.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.test.api;
+
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.security.auth.KafkaPrincipal;
+import org.apache.kafka.common.test.ClusterInstance;
+import org.apache.kafka.common.test.TestUtils;
+import org.apache.kafka.common.test.api.ClusterConfigProperty;
+import org.apache.kafka.common.test.api.ClusterTest;
+import org.apache.kafka.common.test.api.ClusterTestDefaults;
+import org.apache.kafka.common.test.api.Type;
+import org.apache.kafka.common.test.junit.ClusterTestExtensions;
+import org.apache.kafka.server.config.QuotaConfig;
+import org.apache.kafka.server.quota.ClientQuotaCallback;
+import org.apache.kafka.server.quota.ClientQuotaEntity;
+import org.apache.kafka.server.quota.ClientQuotaType;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@ClusterTestDefaults(controllers = 3, 
+    types = {Type.KRAFT},
+    serverProperties = {
+        @ClusterConfigProperty(id = 3000, key = 
QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = 
"kafka.test.api.CustomQuotaCallbackTest$CustomQuotaCallback"),
+        @ClusterConfigProperty(id = 3001, key = 
QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = 
"kafka.test.api.CustomQuotaCallbackTest$CustomQuotaCallback"),
+        @ClusterConfigProperty(id = 3002, key = 
QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, value = 
"kafka.test.api.CustomQuotaCallbackTest$CustomQuotaCallback"),
+    }
+)
+@ExtendWith(ClusterTestExtensions.class)
+public class CustomQuotaCallbackTest {
+
+    private final ClusterInstance cluster;
+
+    public CustomQuotaCallbackTest(ClusterInstance clusterInstance) {
+        this.cluster = clusterInstance;
+    }
+
+    @ClusterTest
+    public void testCustomQuotaCallbackWithControllerServer() throws 
InterruptedException {
+        
+        try (Admin admin = cluster.admin(Map.of())) {
+            admin.createTopics(List.of(new NewTopic("topic", 1, (short) 1)));
+            TestUtils.waitForCondition(
+                () -> CustomQuotaCallback.COUNTERS.size() == 3 
+                        && 
CustomQuotaCallback.COUNTERS.values().stream().allMatch(counter -> 
counter.get() > 0), 
+                    "The CustomQuotaCallback not triggered in all controllers. 
"
+            );
+
+//            List<ClientQuotaAlteration> clientQuotaAlterations = List.of(
+//                new ClientQuotaAlteration(new 
org.apache.kafka.common.quota.ClientQuotaEntity(Map.of(CLIENT_ID, 
"testClient")), List.of(
+//                    new 
ClientQuotaAlteration.Op(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 
8000.0),
+//                    new 
ClientQuotaAlteration.Op(QuotaConfig.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, 
8000.0))
+//                )
+//            );
+            // Reset the counters, and we expect the callback to be triggered 
again in all controllers
+            CustomQuotaCallback.COUNTERS.clear();
+//            admin.alterClientQuotas(clientQuotaAlterations, new 
AlterClientQuotasOptions());
+            admin.deleteTopics(List.of("topic"));
+            TestUtils.waitForCondition(
+                () -> CustomQuotaCallback.COUNTERS.size() == 3
+                        && 
CustomQuotaCallback.COUNTERS.values().stream().allMatch(counter -> 
counter.get() > 0), 
+                    "The CustomQuotaCallback not triggered in all controllers. 
"
+            );
+            
+        }
+    }
+
+
+    public static class CustomQuotaCallback implements ClientQuotaCallback {
+
+        public static final Map<String, AtomicInteger> COUNTERS = new 
ConcurrentHashMap<>();
+        private String nodeId;
+
+        @Override
+        public Map<String, String> quotaMetricTags(ClientQuotaType quotaType, 
KafkaPrincipal principal, String clientId) {
+            return Map.of();
+        }
+
+        @Override
+        public Double quotaLimit(ClientQuotaType quotaType, Map<String, 
String> metricTags) {
+            return 0.0;

Review Comment:
   You have to return `Double.MAX_VALUE` ... otherwise, the ops can't get pass 
as they are rejected due to throttle.



-- 
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