chia7712 commented on code in PR #19807: URL: https://github.com/apache/kafka/pull/19807#discussion_r2160070633
########## clients/src/main/java/org/apache/kafka/server/quota/ClientQuotaCallback.java: ########## @@ -44,7 +45,7 @@ public interface ClientQuotaCallback extends Configurable { * @param clientId The client id associated with the request * @return quota metric tags that indicate which other clients share this quota */ - Map<String, String> quotaMetricTags(ClientQuotaType quotaType, KafkaPrincipal principal, String clientId); + LinkedHashMap<String, String> quotaMetricTags(ClientQuotaType quotaType, KafkaPrincipal principal, String clientId); Review Comment: `ClientQuotaCallback` is a public class, so we can't change the interface without KIP. Additionally, why to change it? while you wan to honor the order, you could use `LinkedHashMap` to copy it. Or you could avoid deep copy. ########## server/src/main/java/org/apache/kafka/server/ClientQuotaManager.java: ########## @@ -0,0 +1,853 @@ +/* + * 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 org.apache.kafka.server; + +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.internals.Plugin; +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.Avg; +import org.apache.kafka.common.metrics.stats.CumulativeSum; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.utils.Sanitizer; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.network.Session; +import org.apache.kafka.server.config.ClientQuotaManagerConfig; +import org.apache.kafka.server.quota.ClientQuotaCallback; +import org.apache.kafka.server.quota.ClientQuotaEntity; +import org.apache.kafka.server.quota.ClientQuotaType; +import org.apache.kafka.server.quota.QuotaType; +import org.apache.kafka.server.quota.QuotaUtils; +import org.apache.kafka.server.quota.SensorAccess; +import org.apache.kafka.server.quota.ThrottleCallback; +import org.apache.kafka.server.quota.ThrottledChannel; +import org.apache.kafka.server.util.ShutdownableThread; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.DelayQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Consumer; + +public class ClientQuotaManager { + + private static final int NO_QUOTAS = 0; + private static final int CLIENT_ID_QUOTA_ENABLED = 1; + private static final int USER_QUOTA_ENABLED = 2; + private static final int USER_CLIENT_ID_QUOTA_ENABLED = 4; + private static final int CUSTOM_QUOTAS = 8; // No metric update optimizations are used with custom quotas + + private static final Logger LOG = LoggerFactory.getLogger(ClientQuotaManager.class); + + // Purge sensors after 1 hour of inactivity + private static final int INACTIVE_SENSOR_EXPIRATION_TIME_SECONDS = 3600; + private static final String DEFAULT_NAME = "<default>"; + + public record UserEntity(String sanitizedUser) implements ClientQuotaEntity.ConfigEntity { + + @Override + public ClientQuotaEntity.ConfigEntityType entityType() { + return ClientQuotaEntity.ConfigEntityType.USER; + } + + @Override + public String name() { + return Sanitizer.desanitize(sanitizedUser); + } + + @Override + public String toString() { + return "user " + sanitizedUser; + } + } + public record ClientIdEntity(String clientId) implements ClientQuotaEntity.ConfigEntity { + + @Override + public ClientQuotaEntity.ConfigEntityType entityType() { + return ClientQuotaEntity.ConfigEntityType.CLIENT_ID; + } + + @Override + public String name() { + return clientId; + } + + @Override + public String toString() { + return "client-id " + clientId; + } + } + + public static final ClientQuotaEntity.ConfigEntity DEFAULT_USER_ENTITY = new ClientQuotaEntity.ConfigEntity() { + @Override + public ClientQuotaEntity.ConfigEntityType entityType() { + return ClientQuotaEntity.ConfigEntityType.DEFAULT_USER; + } + + @Override + public String name() { + return DEFAULT_NAME; + } + + @Override + public String toString() { + return "default user"; + } + }; + + public static final ClientQuotaEntity.ConfigEntity DEFAULT_USER_CLIENT_ID = new ClientQuotaEntity.ConfigEntity() { + @Override + public ClientQuotaEntity.ConfigEntityType entityType() { + return ClientQuotaEntity.ConfigEntityType.DEFAULT_CLIENT_ID; + } + @Override + public String name() { + return DEFAULT_NAME; + } + @Override + public String toString() { + return "default client-id"; + } + }; + + public static final KafkaQuotaEntity DEFAULT_CLIENT_ID_QUOTA_ENTITY = Review Comment: those constants could be `private` ########## core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala: ########## @@ -154,9 +155,9 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag val quotaValue = newValue.map(new Quota(_, true)) Review Comment: `newValue` could be changed to use java `Optional`, since it is converted from java type. -- 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