mirror of https://github.com/apache/kafka.git
KAFKA-10163; Throttle Create Topic, Create Partition and Delete Topic Operations (KIP-599, Part I, Broker Changes) (#8933)
This PR implements the broker side changes of KIP-599, except the changes of the Rate implementation which will be addressed separately. The PR changes/introduces the following: - It introduces the protocol changes. - It introduces a new quota manager ControllerMutationQuotaManager which is another specialization of the ClientQuotaManager. - It enforces the quota in the KafkaApis and in the AdminManager. This part handles new and old clients as described in the KIP. Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
This commit is contained in:
parent
194c56fce2
commit
a5ffd1ca44
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* 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.common.errors;
|
||||
|
||||
/**
|
||||
* Exception thrown if an operation on a resource exceeds the throttling quota.
|
||||
*/
|
||||
public class ThrottlingQuotaExceededException extends RetriableException {
|
||||
private int throttleTimeMs = 0;
|
||||
|
||||
public ThrottlingQuotaExceededException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public ThrottlingQuotaExceededException(int throttleTimeMs, String message) {
|
||||
super(message);
|
||||
this.throttleTimeMs = throttleTimeMs;
|
||||
}
|
||||
|
||||
public int throttleTimeMs() {
|
||||
return this.throttleTimeMs;
|
||||
}
|
||||
}
|
|
@ -97,7 +97,6 @@ public final class Sensor {
|
|||
public boolean shouldRecord(final int configId) {
|
||||
return configId == DEBUG.id || configId == this.id;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private final RecordingLevel recordingLevel;
|
||||
|
@ -180,6 +179,15 @@ public final class Sensor {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Record a value at a known time. This method is slightly faster than {@link #record(double)} since it will reuse
|
||||
* the time stamp.
|
||||
* @param value The value we are recording
|
||||
* @param timeMs The current POSIX time in milliseconds
|
||||
* @param checkQuotas Indicate if quota must be enforced or not
|
||||
* @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
|
||||
* bound
|
||||
*/
|
||||
public void record(double value, long timeMs, boolean checkQuotas) {
|
||||
if (shouldRecord()) {
|
||||
recordInternal(value, timeMs, checkQuotas);
|
||||
|
|
|
@ -78,6 +78,7 @@ import org.apache.kafka.common.errors.OffsetNotAvailableException;
|
|||
import org.apache.kafka.common.errors.OffsetOutOfRangeException;
|
||||
import org.apache.kafka.common.errors.OperationNotAttemptedException;
|
||||
import org.apache.kafka.common.errors.OutOfOrderSequenceException;
|
||||
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
|
||||
import org.apache.kafka.common.errors.UnstableOffsetCommitException;
|
||||
import org.apache.kafka.common.errors.PolicyViolationException;
|
||||
import org.apache.kafka.common.errors.PreferredLeaderNotAvailableException;
|
||||
|
@ -321,7 +322,8 @@ public enum Errors {
|
|||
GROUP_SUBSCRIBED_TO_TOPIC(86, "Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it.",
|
||||
GroupSubscribedToTopicException::new),
|
||||
INVALID_RECORD(87, "This record has failed the validation on broker and hence will be rejected.", InvalidRecordException::new),
|
||||
UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new);
|
||||
UNSTABLE_OFFSET_COMMIT(88, "There are unstable offsets that need to be cleared.", UnstableOffsetCommitException::new),
|
||||
THROTTLING_QUOTA_EXCEEDED(89, "The throttling quota has been exceeded.", ThrottlingQuotaExceededException::new);
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(Errors.class);
|
||||
|
||||
|
|
|
@ -22,5 +22,6 @@ package org.apache.kafka.server.quota;
|
|||
public enum ClientQuotaType {
|
||||
PRODUCE,
|
||||
FETCH,
|
||||
REQUEST
|
||||
REQUEST,
|
||||
CONTROLLER_MUTATION
|
||||
}
|
||||
|
|
|
@ -18,8 +18,12 @@
|
|||
"type": "request",
|
||||
"name": "CreatePartitionsRequest",
|
||||
// Version 1 is the same as version 0.
|
||||
//
|
||||
// Version 2 adds flexible version support
|
||||
"validVersions": "0-2",
|
||||
//
|
||||
// Version 3 is identical to version 2 but may return a THROTTLING_QUOTA_EXCEEDED error
|
||||
// in the response if the partitions creation is throttled (KIP-599).
|
||||
"validVersions": "0-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "Topics", "type": "[]CreatePartitionsTopic", "versions": "0+",
|
||||
|
|
|
@ -18,8 +18,12 @@
|
|||
"type": "response",
|
||||
"name": "CreatePartitionsResponse",
|
||||
// Starting in version 1, on quota violation, brokers send out responses before throttling.
|
||||
//
|
||||
// Version 2 adds flexible version support
|
||||
"validVersions": "0-2",
|
||||
//
|
||||
// Version 3 is identical to version 2 but may return a THROTTLING_QUOTA_EXCEEDED error
|
||||
// in the response if the partitions creation is throttled (KIP-599).
|
||||
"validVersions": "0-3",
|
||||
"flexibleVersions": "2+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
|
||||
|
|
|
@ -23,7 +23,10 @@
|
|||
//
|
||||
// Version 5 is the first flexible version.
|
||||
// Version 5 also returns topic configs in the response (KIP-525).
|
||||
"validVersions": "0-5",
|
||||
//
|
||||
// Version 6 is identical to version 5 but may return a THROTTLING_QUOTA_EXCEEDED error
|
||||
// in the response if the topics creation is throttled (KIP-599).
|
||||
"validVersions": "0-6",
|
||||
"flexibleVersions": "5+",
|
||||
"fields": [
|
||||
{ "name": "Topics", "type": "[]CreatableTopic", "versions": "0+",
|
||||
|
|
|
@ -27,7 +27,10 @@
|
|||
//
|
||||
// Version 5 is the first flexible version.
|
||||
// Version 5 also returns topic configs in the response (KIP-525).
|
||||
"validVersions": "0-5",
|
||||
//
|
||||
// Version 6 is identical to version 5 but may return a THROTTLING_QUOTA_EXCEEDED error
|
||||
// in the response if the topics creation is throttled (KIP-599).
|
||||
"validVersions": "0-6",
|
||||
"flexibleVersions": "5+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,
|
||||
|
|
|
@ -20,7 +20,10 @@
|
|||
// Versions 0, 1, 2, and 3 are the same.
|
||||
//
|
||||
// Version 4 is the first flexible version.
|
||||
"validVersions": "0-4",
|
||||
//
|
||||
// Version 5 adds ErrorMessage in the response and may return a THROTTLING_QUOTA_EXCEEDED error
|
||||
// in the response if the topics deletion is throttled (KIP-599).
|
||||
"validVersions": "0-5",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "TopicNames", "type": "[]string", "versions": "0+", "entityType": "topicName",
|
||||
|
|
|
@ -24,7 +24,10 @@
|
|||
// Starting in version 3, a TOPIC_DELETION_DISABLED error code may be returned.
|
||||
//
|
||||
// Version 4 is the first flexible version.
|
||||
"validVersions": "0-4",
|
||||
//
|
||||
// Version 5 adds ErrorMessage in the response and may return a THROTTLING_QUOTA_EXCEEDED error
|
||||
// in the response if the topics deletion is throttled (KIP-599).
|
||||
"validVersions": "0-5",
|
||||
"flexibleVersions": "4+",
|
||||
"fields": [
|
||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
|
||||
|
@ -34,7 +37,9 @@
|
|||
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName",
|
||||
"about": "The topic name" },
|
||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||
"about": "The deletion error, or 0 if the deletion succeeded." }
|
||||
"about": "The deletion error, or 0 if the deletion succeeded." },
|
||||
{ "name": "ErrorMessage", "type": "string", "versions": "5+", "nullableVersions": "5+", "ignorable": true,
|
||||
"about": "The error message, or null if there was no error." }
|
||||
]}
|
||||
]
|
||||
}
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.kafka.common.acl.AclBindingFilter;
|
|||
import org.apache.kafka.common.acl.AclOperation;
|
||||
import org.apache.kafka.common.acl.AclPermissionType;
|
||||
import org.apache.kafka.common.config.ConfigResource;
|
||||
import org.apache.kafka.common.errors.InvalidTopicException;
|
||||
import org.apache.kafka.common.errors.NotCoordinatorException;
|
||||
import org.apache.kafka.common.errors.NotEnoughReplicasException;
|
||||
import org.apache.kafka.common.errors.SecurityDisabledException;
|
||||
|
@ -327,28 +326,24 @@ public class RequestResponseTest {
|
|||
checkResponse(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE, 2, true);
|
||||
checkResponse(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE, 3, true);
|
||||
|
||||
checkRequest(createCreateTopicRequest(0), true);
|
||||
checkErrorResponse(createCreateTopicRequest(0), unknownServerException, true);
|
||||
checkResponse(createCreateTopicResponse(), 0, true);
|
||||
checkRequest(createCreateTopicRequest(1), true);
|
||||
checkErrorResponse(createCreateTopicRequest(1), unknownServerException, true);
|
||||
checkResponse(createCreateTopicResponse(), 1, true);
|
||||
checkRequest(createCreateTopicRequest(2), true);
|
||||
checkErrorResponse(createCreateTopicRequest(2), unknownServerException, true);
|
||||
checkResponse(createCreateTopicResponse(), 2, true);
|
||||
checkRequest(createCreateTopicRequest(3), true);
|
||||
checkErrorResponse(createCreateTopicRequest(3), unknownServerException, true);
|
||||
checkResponse(createCreateTopicResponse(), 3, true);
|
||||
checkRequest(createCreateTopicRequest(4), true);
|
||||
checkErrorResponse(createCreateTopicRequest(4), unknownServerException, true);
|
||||
checkResponse(createCreateTopicResponse(), 4, true);
|
||||
checkRequest(createCreateTopicRequest(5), true);
|
||||
checkErrorResponse(createCreateTopicRequest(5), unknownServerException, true);
|
||||
checkResponse(createCreateTopicResponse(), 5, true);
|
||||
for (int v = ApiKeys.CREATE_TOPICS.oldestVersion(); v <= ApiKeys.CREATE_TOPICS.latestVersion(); v++) {
|
||||
checkRequest(createCreateTopicRequest(v), true);
|
||||
checkErrorResponse(createCreateTopicRequest(v), unknownServerException, true);
|
||||
checkResponse(createCreateTopicResponse(), v, true);
|
||||
}
|
||||
|
||||
checkRequest(createDeleteTopicsRequest(), true);
|
||||
checkErrorResponse(createDeleteTopicsRequest(), unknownServerException, true);
|
||||
checkResponse(createDeleteTopicsResponse(), 0, true);
|
||||
for (int v = ApiKeys.DELETE_TOPICS.oldestVersion(); v <= ApiKeys.DELETE_TOPICS.latestVersion(); v++) {
|
||||
checkRequest(createDeleteTopicsRequest(v), true);
|
||||
checkErrorResponse(createDeleteTopicsRequest(v), unknownServerException, true);
|
||||
checkResponse(createDeleteTopicsResponse(), v, true);
|
||||
}
|
||||
|
||||
for (int v = ApiKeys.CREATE_PARTITIONS.oldestVersion(); v <= ApiKeys.CREATE_PARTITIONS.latestVersion(); v++) {
|
||||
checkRequest(createCreatePartitionsRequest(v), true);
|
||||
checkRequest(createCreatePartitionsRequestWithAssignments(v), false);
|
||||
checkErrorResponse(createCreatePartitionsRequest(v), unknownServerException, true);
|
||||
checkResponse(createCreatePartitionsResponse(), v, true);
|
||||
}
|
||||
|
||||
checkRequest(createInitPidRequest(), true);
|
||||
checkErrorResponse(createInitPidRequest(), unknownServerException, true);
|
||||
|
@ -454,10 +449,6 @@ public class RequestResponseTest {
|
|||
checkErrorResponse(createDescribeConfigsRequest(1), unknownServerException, true);
|
||||
checkResponse(createDescribeConfigsResponse((short) 1), 1, false);
|
||||
checkDescribeConfigsResponseVersions();
|
||||
checkRequest(createCreatePartitionsRequest(), true);
|
||||
checkRequest(createCreatePartitionsRequestWithAssignments(), false);
|
||||
checkErrorResponse(createCreatePartitionsRequest(), new InvalidTopicException(), true);
|
||||
checkResponse(createCreatePartitionsResponse(), 0, true);
|
||||
checkRequest(createCreateTokenRequest(), true);
|
||||
checkErrorResponse(createCreateTokenRequest(), unknownServerException, true);
|
||||
checkResponse(createCreateTokenResponse(), 0, true);
|
||||
|
@ -1602,23 +1593,23 @@ public class RequestResponseTest {
|
|||
}
|
||||
|
||||
private CreateTopicsRequest createCreateTopicRequest(int version, boolean validateOnly) {
|
||||
CreateTopicsRequestData data = new CreateTopicsRequestData().
|
||||
setTimeoutMs(123).
|
||||
setValidateOnly(validateOnly);
|
||||
data.topics().add(new CreatableTopic().
|
||||
setNumPartitions(3).
|
||||
setReplicationFactor((short) 5));
|
||||
CreateTopicsRequestData data = new CreateTopicsRequestData()
|
||||
.setTimeoutMs(123)
|
||||
.setValidateOnly(validateOnly);
|
||||
data.topics().add(new CreatableTopic()
|
||||
.setNumPartitions(3)
|
||||
.setReplicationFactor((short) 5));
|
||||
|
||||
CreatableTopic topic2 = new CreatableTopic();
|
||||
data.topics().add(topic2);
|
||||
topic2.assignments().add(new CreatableReplicaAssignment().
|
||||
setPartitionIndex(0).
|
||||
setBrokerIds(Arrays.asList(1, 2, 3)));
|
||||
topic2.assignments().add(new CreatableReplicaAssignment().
|
||||
setPartitionIndex(1).
|
||||
setBrokerIds(Arrays.asList(2, 3, 4)));
|
||||
topic2.configs().add(new CreateableTopicConfig().
|
||||
setName("config1").setValue("value1"));
|
||||
topic2.assignments().add(new CreatableReplicaAssignment()
|
||||
.setPartitionIndex(0)
|
||||
.setBrokerIds(Arrays.asList(1, 2, 3)));
|
||||
topic2.assignments().add(new CreatableReplicaAssignment()
|
||||
.setPartitionIndex(1)
|
||||
.setBrokerIds(Arrays.asList(2, 3, 4)));
|
||||
topic2.configs().add(new CreateableTopicConfig()
|
||||
.setName("config1").setValue("value1"));
|
||||
|
||||
return new CreateTopicsRequest.Builder(data).build((short) version);
|
||||
}
|
||||
|
@ -1644,21 +1635,23 @@ public class RequestResponseTest {
|
|||
return new CreateTopicsResponse(data);
|
||||
}
|
||||
|
||||
private DeleteTopicsRequest createDeleteTopicsRequest() {
|
||||
return new DeleteTopicsRequest.Builder(
|
||||
new DeleteTopicsRequestData()
|
||||
.setTopicNames(Arrays.asList("my_t1", "my_t2"))
|
||||
.setTimeoutMs(1000)).build();
|
||||
private DeleteTopicsRequest createDeleteTopicsRequest(int version) {
|
||||
return new DeleteTopicsRequest.Builder(new DeleteTopicsRequestData()
|
||||
.setTopicNames(Arrays.asList("my_t1", "my_t2"))
|
||||
.setTimeoutMs(1000)
|
||||
).build((short) version);
|
||||
}
|
||||
|
||||
private DeleteTopicsResponse createDeleteTopicsResponse() {
|
||||
DeleteTopicsResponseData data = new DeleteTopicsResponseData();
|
||||
data.responses().add(new DeletableTopicResult()
|
||||
.setName("t1")
|
||||
.setErrorCode(Errors.INVALID_TOPIC_EXCEPTION.code()));
|
||||
.setName("t1")
|
||||
.setErrorCode(Errors.INVALID_TOPIC_EXCEPTION.code())
|
||||
.setErrorMessage("Error Message"));
|
||||
data.responses().add(new DeletableTopicResult()
|
||||
.setName("t2")
|
||||
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code()));
|
||||
.setName("t2")
|
||||
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code())
|
||||
.setErrorMessage("Error Message"));
|
||||
return new DeleteTopicsResponse(data);
|
||||
}
|
||||
|
||||
|
@ -2019,7 +2012,7 @@ public class RequestResponseTest {
|
|||
return new AlterConfigsResponse(data);
|
||||
}
|
||||
|
||||
private CreatePartitionsRequest createCreatePartitionsRequest() {
|
||||
private CreatePartitionsRequest createCreatePartitionsRequest(int version) {
|
||||
List<CreatePartitionsTopic> topics = new LinkedList<>();
|
||||
topics.add(new CreatePartitionsTopic()
|
||||
.setName("my_topic")
|
||||
|
@ -2034,10 +2027,11 @@ public class RequestResponseTest {
|
|||
.setTimeoutMs(0)
|
||||
.setValidateOnly(false)
|
||||
.setTopics(topics);
|
||||
return new CreatePartitionsRequest(data, (short) 0);
|
||||
|
||||
return new CreatePartitionsRequest(data, (short) version);
|
||||
}
|
||||
|
||||
private CreatePartitionsRequest createCreatePartitionsRequestWithAssignments() {
|
||||
private CreatePartitionsRequest createCreatePartitionsRequestWithAssignments(int version) {
|
||||
List<CreatePartitionsTopic> topics = new LinkedList<>();
|
||||
CreatePartitionsAssignment myTopicAssignment = new CreatePartitionsAssignment()
|
||||
.setBrokerIds(Collections.singletonList(2));
|
||||
|
@ -2060,7 +2054,8 @@ public class RequestResponseTest {
|
|||
.setTimeoutMs(0)
|
||||
.setValidateOnly(false)
|
||||
.setTopics(topics);
|
||||
return new CreatePartitionsRequest(data, (short) 0);
|
||||
|
||||
return new CreatePartitionsRequest(data, (short) version);
|
||||
}
|
||||
|
||||
private CreatePartitionsResponse createCreatePartitionsResponse() {
|
||||
|
|
|
@ -181,11 +181,11 @@ abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: Stri
|
|||
|
||||
def closeAllFetchers(): Unit = {
|
||||
lock synchronized {
|
||||
for ( (_, fetcher) <- fetcherThreadMap) {
|
||||
for ((_, fetcher) <- fetcherThreadMap) {
|
||||
fetcher.initiateShutdown()
|
||||
}
|
||||
|
||||
for ( (_, fetcher) <- fetcherThreadMap) {
|
||||
for ((_, fetcher) <- fetcherThreadMap) {
|
||||
fetcher.shutdown()
|
||||
}
|
||||
fetcherThreadMap.clear()
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.kafka.clients.admin.AlterConfigOp
|
|||
import org.apache.kafka.clients.admin.AlterConfigOp.OpType
|
||||
import org.apache.kafka.common.config.ConfigDef.ConfigKey
|
||||
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, LogLevelConfig}
|
||||
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException
|
||||
import org.apache.kafka.common.errors.{ApiException, InvalidConfigurationException, InvalidPartitionsException, InvalidReplicaAssignmentException, InvalidRequestException, ReassignmentInProgressException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedVersionException}
|
||||
import org.apache.kafka.common.internals.Topic
|
||||
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
|
||||
|
@ -38,7 +39,6 @@ import org.apache.kafka.common.message.CreateTopicsResponseData.{CreatableTopicC
|
|||
import org.apache.kafka.common.message.DescribeConfigsResponseData
|
||||
import org.apache.kafka.common.message.DescribeConfigsRequestData.DescribeConfigsResource
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.server.policy.{AlterConfigPolicy, CreateTopicPolicy}
|
||||
import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
|
@ -81,6 +81,52 @@ class AdminManager(val config: KafkaConfig,
|
|||
debug(s"Request key ${key.keyLabel} unblocked $completed topic requests.")
|
||||
}
|
||||
|
||||
private def validateTopicCreatePolicy(topic: CreatableTopic,
|
||||
resolvedNumPartitions: Int,
|
||||
resolvedReplicationFactor: Short,
|
||||
assignments: Map[Int, Seq[Int]]): Unit = {
|
||||
createTopicPolicy.foreach { policy =>
|
||||
// Use `null` for unset fields in the public API
|
||||
val numPartitions: java.lang.Integer =
|
||||
if (topic.assignments().isEmpty) resolvedNumPartitions else null
|
||||
val replicationFactor: java.lang.Short =
|
||||
if (topic.assignments().isEmpty) resolvedReplicationFactor else null
|
||||
val javaAssignments = if (topic.assignments().isEmpty) {
|
||||
null
|
||||
} else {
|
||||
assignments.map { case (k, v) =>
|
||||
(k: java.lang.Integer) -> v.map(i => i: java.lang.Integer).asJava
|
||||
}.asJava
|
||||
}
|
||||
val javaConfigs = new java.util.HashMap[String, String]
|
||||
topic.configs.forEach(config => javaConfigs.put(config.name, config.value))
|
||||
policy.validate(new RequestMetadata(topic.name, numPartitions, replicationFactor,
|
||||
javaAssignments, javaConfigs))
|
||||
}
|
||||
}
|
||||
|
||||
private def maybePopulateMetadataAndConfigs(metadataAndConfigs: Map[String, CreatableTopicResult],
|
||||
topicName: String,
|
||||
configs: Properties,
|
||||
assignments: Map[Int, Seq[Int]]): Unit = {
|
||||
metadataAndConfigs.get(topicName).foreach { result =>
|
||||
val logConfig = LogConfig.fromProps(KafkaServer.copyKafkaConfigToLog(config), configs)
|
||||
val createEntry = createTopicConfigEntry(logConfig, configs, includeSynonyms = false, includeDocumentation = false)(_, _)
|
||||
val topicConfigs = logConfig.values.asScala.map { case (k, v) =>
|
||||
val entry = createEntry(k, v)
|
||||
new CreatableTopicConfigs()
|
||||
.setName(k)
|
||||
.setValue(entry.value)
|
||||
.setIsSensitive(entry.isSensitive)
|
||||
.setReadOnly(entry.readOnly)
|
||||
.setConfigSource(entry.configSource)
|
||||
}.toList.asJava
|
||||
result.setConfigs(topicConfigs)
|
||||
result.setNumPartitions(assignments.size)
|
||||
result.setReplicationFactor(assignments(0).size.toShort)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create topics and wait until the topics have been completely created.
|
||||
* The callback function will be triggered either when timeout, error or the topics are created.
|
||||
|
@ -88,7 +134,8 @@ class AdminManager(val config: KafkaConfig,
|
|||
def createTopics(timeout: Int,
|
||||
validateOnly: Boolean,
|
||||
toCreate: Map[String, CreatableTopic],
|
||||
includeConfigsAndMetatadata: Map[String, CreatableTopicResult],
|
||||
includeConfigsAndMetadata: Map[String, CreatableTopicResult],
|
||||
controllerMutationQuota: ControllerMutationQuota,
|
||||
responseCallback: Map[String, ApiError] => Unit): Unit = {
|
||||
|
||||
// 1. map over topics creating assignment and calling zookeeper
|
||||
|
@ -102,12 +149,6 @@ class AdminManager(val config: KafkaConfig,
|
|||
if (nullConfigs.nonEmpty)
|
||||
throw new InvalidRequestException(s"Null value not supported for topic configs : ${nullConfigs.mkString(",")}")
|
||||
|
||||
val configs = new Properties()
|
||||
topic.configs.forEach { entry =>
|
||||
configs.setProperty(entry.name, entry.value)
|
||||
}
|
||||
LogConfig.validate(configs)
|
||||
|
||||
if ((topic.numPartitions != NO_NUM_PARTITIONS || topic.replicationFactor != NO_REPLICATION_FACTOR)
|
||||
&& !topic.assignments().isEmpty) {
|
||||
throw new InvalidRequestException("Both numPartitions or replicationFactor and replicasAssignments were set. " +
|
||||
|
@ -119,7 +160,7 @@ class AdminManager(val config: KafkaConfig,
|
|||
val resolvedReplicationFactor = if (topic.replicationFactor == NO_REPLICATION_FACTOR)
|
||||
defaultReplicationFactor else topic.replicationFactor
|
||||
|
||||
val assignments = if (topic.assignments().isEmpty) {
|
||||
val assignments = if (topic.assignments.isEmpty) {
|
||||
AdminUtils.assignReplicasToBrokers(
|
||||
brokers, resolvedNumPartitions, resolvedReplicationFactor)
|
||||
} else {
|
||||
|
@ -133,76 +174,47 @@ class AdminManager(val config: KafkaConfig,
|
|||
}
|
||||
trace(s"Assignments for topic $topic are $assignments ")
|
||||
|
||||
createTopicPolicy match {
|
||||
case Some(policy) =>
|
||||
adminZkClient.validateTopicCreate(topic.name, assignments, configs)
|
||||
val configs = new Properties()
|
||||
topic.configs.forEach(entry => configs.setProperty(entry.name, entry.value))
|
||||
adminZkClient.validateTopicCreate(topic.name, assignments, configs)
|
||||
validateTopicCreatePolicy(topic, resolvedNumPartitions, resolvedReplicationFactor, assignments)
|
||||
|
||||
// Use `null` for unset fields in the public API
|
||||
val numPartitions: java.lang.Integer =
|
||||
if (topic.assignments().isEmpty) resolvedNumPartitions else null
|
||||
val replicationFactor: java.lang.Short =
|
||||
if (topic.assignments().isEmpty) resolvedReplicationFactor else null
|
||||
val javaAssignments = if (topic.assignments().isEmpty) {
|
||||
null
|
||||
} else {
|
||||
assignments.map { case (k, v) =>
|
||||
(k: java.lang.Integer) -> v.map(i => i: java.lang.Integer).asJava
|
||||
}.asJava
|
||||
}
|
||||
val javaConfigs = new java.util.HashMap[String, String]
|
||||
topic.configs.forEach(config => javaConfigs.put(config.name, config.value))
|
||||
policy.validate(new RequestMetadata(topic.name, numPartitions, replicationFactor,
|
||||
javaAssignments, javaConfigs))
|
||||
// For responses with DescribeConfigs permission, populate metadata and configs. It is
|
||||
// safe to populate it before creating the topic because the values are unset if the
|
||||
// creation fails.
|
||||
maybePopulateMetadataAndConfigs(includeConfigsAndMetadata, topic.name, configs, assignments)
|
||||
|
||||
if (!validateOnly)
|
||||
adminZkClient.createTopicWithAssignment(topic.name, configs, assignments)
|
||||
|
||||
case None =>
|
||||
if (validateOnly)
|
||||
adminZkClient.validateTopicCreate(topic.name, assignments, configs)
|
||||
else
|
||||
adminZkClient.createTopicWithAssignment(topic.name, configs, assignments)
|
||||
if (validateOnly) {
|
||||
CreatePartitionsMetadata(topic.name, assignments.keySet)
|
||||
} else {
|
||||
controllerMutationQuota.record(assignments.size)
|
||||
adminZkClient.createTopicWithAssignment(topic.name, configs, assignments, validate = false)
|
||||
CreatePartitionsMetadata(topic.name, assignments.keySet)
|
||||
}
|
||||
|
||||
// For responses with DescribeConfigs permission, populate metadata and configs
|
||||
includeConfigsAndMetatadata.get(topic.name).foreach { result =>
|
||||
val logConfig = LogConfig.fromProps(KafkaServer.copyKafkaConfigToLog(config), configs)
|
||||
val createEntry = createTopicConfigEntry(logConfig, configs, includeSynonyms = false, includeDocumentation = false)(_, _)
|
||||
val topicConfigs = logConfig.values.asScala.map { case (k, v) =>
|
||||
val entry = createEntry(k, v)
|
||||
new CreatableTopicConfigs()
|
||||
.setName(k)
|
||||
.setValue(entry.value)
|
||||
.setIsSensitive(entry.isSensitive)
|
||||
.setReadOnly(entry.readOnly)
|
||||
.setConfigSource(entry.configSource)
|
||||
}.toList.asJava
|
||||
result.setConfigs(topicConfigs)
|
||||
result.setNumPartitions(assignments.size)
|
||||
result.setReplicationFactor(assignments(0).size.toShort)
|
||||
}
|
||||
CreatePartitionsMetadata(topic.name, assignments.keySet, ApiError.NONE)
|
||||
} catch {
|
||||
// Log client errors at a lower level than unexpected exceptions
|
||||
case e: TopicExistsException =>
|
||||
debug(s"Topic creation failed since topic '${topic.name}' already exists.", e)
|
||||
CreatePartitionsMetadata(topic.name, Set.empty, ApiError.fromThrowable(e))
|
||||
CreatePartitionsMetadata(topic.name, e)
|
||||
case e: ThrottlingQuotaExceededException =>
|
||||
debug(s"Topic creation not allowed because quota is violated. Delay time: ${e.throttleTimeMs}")
|
||||
CreatePartitionsMetadata(topic.name, e)
|
||||
case e: ApiException =>
|
||||
info(s"Error processing create topic request $topic", e)
|
||||
CreatePartitionsMetadata(topic.name, Set.empty, ApiError.fromThrowable(e))
|
||||
CreatePartitionsMetadata(topic.name, e)
|
||||
case e: ConfigException =>
|
||||
info(s"Error processing create topic request $topic", e)
|
||||
CreatePartitionsMetadata(topic.name, Set.empty, ApiError.fromThrowable(new InvalidConfigurationException(e.getMessage, e.getCause)))
|
||||
CreatePartitionsMetadata(topic.name, new InvalidConfigurationException(e.getMessage, e.getCause))
|
||||
case e: Throwable =>
|
||||
error(s"Error processing create topic request $topic", e)
|
||||
CreatePartitionsMetadata(topic.name, Set.empty, ApiError.fromThrowable(e))
|
||||
CreatePartitionsMetadata(topic.name, e)
|
||||
}).toBuffer
|
||||
|
||||
// 2. if timeout <= 0, validateOnly or no topics can proceed return immediately
|
||||
if (timeout <= 0 || validateOnly || !metadata.exists(_.error.is(Errors.NONE))) {
|
||||
val results = metadata.map { createTopicMetadata =>
|
||||
// ignore topics that already have errors
|
||||
if (createTopicMetadata.error.isSuccess() && !validateOnly) {
|
||||
if (createTopicMetadata.error.isSuccess && !validateOnly) {
|
||||
(createTopicMetadata.topic, new ApiError(Errors.REQUEST_TIMED_OUT, null))
|
||||
} else {
|
||||
(createTopicMetadata.topic, createTopicMetadata.error)
|
||||
|
@ -211,8 +223,9 @@ class AdminManager(val config: KafkaConfig,
|
|||
responseCallback(results)
|
||||
} else {
|
||||
// 3. else pass the assignments and errors to the delayed operation and set the keys
|
||||
val delayedCreate = new DelayedCreatePartitions(timeout, metadata, this, responseCallback)
|
||||
val delayedCreateKeys = toCreate.values.map(topic => new TopicKey(topic.name)).toBuffer
|
||||
val delayedCreate = new DelayedCreatePartitions(timeout, metadata, this,
|
||||
responseCallback)
|
||||
val delayedCreateKeys = toCreate.values.map(topic => TopicKey(topic.name)).toBuffer
|
||||
// try to complete the request immediately, otherwise put it into the purgatory
|
||||
topicPurgatory.tryCompleteElseWatch(delayedCreate, delayedCreateKeys)
|
||||
}
|
||||
|
@ -224,20 +237,24 @@ class AdminManager(val config: KafkaConfig,
|
|||
*/
|
||||
def deleteTopics(timeout: Int,
|
||||
topics: Set[String],
|
||||
controllerMutationQuota: ControllerMutationQuota,
|
||||
responseCallback: Map[String, Errors] => Unit): Unit = {
|
||||
|
||||
// 1. map over topics calling the asynchronous delete
|
||||
val metadata = topics.map { topic =>
|
||||
try {
|
||||
controllerMutationQuota.record(metadataCache.numPartitions(topic).getOrElse(0).toDouble)
|
||||
adminZkClient.deleteTopic(topic)
|
||||
DeleteTopicMetadata(topic, Errors.NONE)
|
||||
} catch {
|
||||
case _: TopicAlreadyMarkedForDeletionException =>
|
||||
// swallow the exception, and still track deletion allowing multiple calls to wait for deletion
|
||||
DeleteTopicMetadata(topic, Errors.NONE)
|
||||
case e: ThrottlingQuotaExceededException =>
|
||||
debug(s"Topic deletion not allowed because quota is violated. Delay time: ${e.throttleTimeMs}")
|
||||
DeleteTopicMetadata(topic, e)
|
||||
case e: Throwable =>
|
||||
error(s"Error processing delete topic request for topic $topic", e)
|
||||
DeleteTopicMetadata(topic, Errors.forException(e))
|
||||
DeleteTopicMetadata(topic, e)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -255,7 +272,7 @@ class AdminManager(val config: KafkaConfig,
|
|||
} else {
|
||||
// 3. else pass the topics and errors to the delayed operation and set the keys
|
||||
val delayedDelete = new DelayedDeleteTopics(timeout, metadata.toSeq, this, responseCallback)
|
||||
val delayedDeleteKeys = topics.map(new TopicKey(_)).toSeq
|
||||
val delayedDeleteKeys = topics.map(TopicKey).toSeq
|
||||
// try to complete the request immediately, otherwise put it into the purgatory
|
||||
topicPurgatory.tryCompleteElseWatch(delayedDelete, delayedDeleteKeys)
|
||||
}
|
||||
|
@ -264,15 +281,15 @@ class AdminManager(val config: KafkaConfig,
|
|||
def createPartitions(timeout: Int,
|
||||
newPartitions: Seq[CreatePartitionsTopic],
|
||||
validateOnly: Boolean,
|
||||
listenerName: ListenerName,
|
||||
controllerMutationQuota: ControllerMutationQuota,
|
||||
callback: Map[String, ApiError] => Unit): Unit = {
|
||||
|
||||
val allBrokers = adminZkClient.getBrokerMetadatas()
|
||||
val allBrokerIds = allBrokers.map(_.id)
|
||||
|
||||
// 1. map over topics creating assignment and calling AdminUtils
|
||||
val metadata = newPartitions.map { newPartition =>
|
||||
val topic = newPartition.name
|
||||
|
||||
try {
|
||||
val existingAssignment = zkClient.getFullReplicaAssignmentForTopics(immutable.Set(topic)).map {
|
||||
case (topicPartition, assignment) =>
|
||||
|
@ -296,34 +313,44 @@ class AdminManager(val config: KafkaConfig,
|
|||
throw new InvalidPartitionsException(s"Topic already has $oldNumPartitions partitions.")
|
||||
}
|
||||
|
||||
val newPartitionsAssignment = Option(newPartition.assignments)
|
||||
.map { assignmentMap =>
|
||||
val assignments = assignmentMap.asScala.map {
|
||||
createPartitionAssignment => createPartitionAssignment.brokerIds.asScala.map(_.toInt)
|
||||
}
|
||||
val unknownBrokers = assignments.flatten.toSet -- allBrokerIds
|
||||
if (unknownBrokers.nonEmpty)
|
||||
throw new InvalidReplicaAssignmentException(
|
||||
s"Unknown broker(s) in replica assignment: ${unknownBrokers.mkString(", ")}.")
|
||||
val newPartitionsAssignment = Option(newPartition.assignments).map { assignmentMap =>
|
||||
val assignments = assignmentMap.asScala.map {
|
||||
createPartitionAssignment => createPartitionAssignment.brokerIds.asScala.map(_.toInt)
|
||||
}
|
||||
val unknownBrokers = assignments.flatten.toSet -- allBrokerIds
|
||||
if (unknownBrokers.nonEmpty)
|
||||
throw new InvalidReplicaAssignmentException(
|
||||
s"Unknown broker(s) in replica assignment: ${unknownBrokers.mkString(", ")}.")
|
||||
|
||||
if (assignments.size != numPartitionsIncrement)
|
||||
throw new InvalidReplicaAssignmentException(
|
||||
s"Increasing the number of partitions by $numPartitionsIncrement " +
|
||||
s"but ${assignments.size} assignments provided.")
|
||||
if (assignments.size != numPartitionsIncrement)
|
||||
throw new InvalidReplicaAssignmentException(
|
||||
s"Increasing the number of partitions by $numPartitionsIncrement " +
|
||||
s"but ${assignments.size} assignments provided.")
|
||||
|
||||
assignments.zipWithIndex.map { case (replicas, index) =>
|
||||
existingAssignment.size + index -> replicas
|
||||
}.toMap
|
||||
assignments.zipWithIndex.map { case (replicas, index) =>
|
||||
existingAssignment.size + index -> replicas
|
||||
}.toMap
|
||||
}
|
||||
|
||||
val updatedReplicaAssignment = adminZkClient.addPartitions(topic, existingAssignment, allBrokers,
|
||||
newPartition.count, newPartitionsAssignment, validateOnly = validateOnly)
|
||||
CreatePartitionsMetadata(topic, updatedReplicaAssignment.keySet, ApiError.NONE)
|
||||
val assignmentForNewPartitions = adminZkClient.createNewPartitionsAssignment(
|
||||
topic, existingAssignment, allBrokers, newPartition.count, newPartitionsAssignment)
|
||||
|
||||
if (validateOnly) {
|
||||
CreatePartitionsMetadata(topic, (existingAssignment ++ assignmentForNewPartitions).keySet)
|
||||
} else {
|
||||
controllerMutationQuota.record(numPartitionsIncrement)
|
||||
val updatedReplicaAssignment = adminZkClient.createPartitionsWithAssignment(
|
||||
topic, existingAssignment, assignmentForNewPartitions)
|
||||
CreatePartitionsMetadata(topic, updatedReplicaAssignment.keySet)
|
||||
}
|
||||
} catch {
|
||||
case e: AdminOperationException =>
|
||||
CreatePartitionsMetadata(topic, Set.empty, ApiError.fromThrowable(e))
|
||||
CreatePartitionsMetadata(topic, e)
|
||||
case e: ThrottlingQuotaExceededException =>
|
||||
debug(s"Partition(s) creation not allowed because quota is violated. Delay time: ${e.throttleTimeMs}")
|
||||
CreatePartitionsMetadata(topic, e)
|
||||
case e: ApiException =>
|
||||
CreatePartitionsMetadata(topic, Set.empty, ApiError.fromThrowable(e))
|
||||
CreatePartitionsMetadata(topic, e)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -347,7 +374,9 @@ class AdminManager(val config: KafkaConfig,
|
|||
}
|
||||
}
|
||||
|
||||
def describeConfigs(resourceToConfigNames: List[DescribeConfigsResource], includeSynonyms: Boolean, includeDocumentation: Boolean): List[DescribeConfigsResponseData.DescribeConfigsResult] = {
|
||||
def describeConfigs(resourceToConfigNames: List[DescribeConfigsResource],
|
||||
includeSynonyms: Boolean,
|
||||
includeDocumentation: Boolean): List[DescribeConfigsResponseData.DescribeConfigsResult] = {
|
||||
resourceToConfigNames.map { case resource =>
|
||||
|
||||
def allConfigs(config: AbstractConfig) = {
|
||||
|
|
|
@ -44,30 +44,24 @@ case class ClientSensors(metricTags: Map[String, String], quotaSensor: Sensor, t
|
|||
|
||||
/**
|
||||
* Configuration settings for quota management
|
||||
* @param quotaBytesPerSecondDefault The default bytes per second quota allocated to any client-id if
|
||||
* @param quotaDefault The default allocated to any client-id if
|
||||
* dynamic defaults or user quotas are not set
|
||||
* @param numQuotaSamples The number of samples to retain in memory
|
||||
* @param quotaWindowSizeSeconds The time span of each sample
|
||||
*
|
||||
*/
|
||||
case class ClientQuotaManagerConfig(quotaBytesPerSecondDefault: Long =
|
||||
ClientQuotaManagerConfig.QuotaBytesPerSecondDefault,
|
||||
case class ClientQuotaManagerConfig(quotaDefault: Long =
|
||||
ClientQuotaManagerConfig.QuotaDefault,
|
||||
numQuotaSamples: Int =
|
||||
ClientQuotaManagerConfig.DefaultNumQuotaSamples,
|
||||
quotaWindowSizeSeconds: Int =
|
||||
ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds)
|
||||
|
||||
object ClientQuotaManagerConfig {
|
||||
val QuotaBytesPerSecondDefault = Long.MaxValue
|
||||
val QuotaDefault = Long.MaxValue
|
||||
// Always have 10 whole windows + 1 current window
|
||||
val DefaultNumQuotaSamples = 11
|
||||
val DefaultQuotaWindowSizeSeconds = 1
|
||||
// Purge sensors after 1 hour of inactivity
|
||||
val InactiveSensorExpirationTimeSeconds = 3600
|
||||
val QuotaRequestPercentDefault = Int.MaxValue.toDouble
|
||||
val NanosToPercentagePerSecond = 100.0 / TimeUnit.SECONDS.toNanos(1)
|
||||
|
||||
val UnlimitedQuota = Quota.upperBound(Long.MaxValue)
|
||||
}
|
||||
|
||||
object QuotaTypes {
|
||||
|
@ -79,6 +73,9 @@ object QuotaTypes {
|
|||
}
|
||||
|
||||
object ClientQuotaManager {
|
||||
// Purge sensors after 1 hour of inactivity
|
||||
val InactiveSensorExpirationTimeSeconds = 3600
|
||||
|
||||
val DefaultClientIdQuotaEntity = KafkaQuotaEntity(None, Some(DefaultClientIdEntity))
|
||||
val DefaultUserQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), None)
|
||||
val DefaultUserClientIdQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), Some(DefaultClientIdEntity))
|
||||
|
@ -111,10 +108,12 @@ object ClientQuotaManager {
|
|||
clientIdEntity: Option[ClientQuotaEntity.ConfigEntity]) extends ClientQuotaEntity {
|
||||
override def configEntities: util.List[ClientQuotaEntity.ConfigEntity] =
|
||||
(userEntity.toList ++ clientIdEntity.toList).asJava
|
||||
|
||||
def sanitizedUser: String = userEntity.map {
|
||||
case entity: UserEntity => entity.sanitizedUser
|
||||
case DefaultUserEntity => ConfigEntityName.Default
|
||||
}.getOrElse("")
|
||||
|
||||
def clientId: String = clientIdEntity.map(_.name).getOrElse("")
|
||||
|
||||
override def toString: String = {
|
||||
|
@ -128,6 +127,32 @@ object ClientQuotaManager {
|
|||
val User = "user"
|
||||
val ClientId = "client-id"
|
||||
}
|
||||
|
||||
/**
|
||||
* This calculates the amount of time needed to bring the metric within quota
|
||||
* assuming that no new metrics are recorded.
|
||||
*
|
||||
* Basically, if O is the observed rate and T is the target rate over a window of W, to bring O down to T,
|
||||
* we need to add a delay of X to W such that O * W / (W + X) = T.
|
||||
* Solving for X, we get X = (O - T)/T * W.
|
||||
*/
|
||||
def throttleTime(e: QuotaViolationException, timeMs: Long): Long = {
|
||||
val difference = e.value - e.bound
|
||||
// Use the precise window used by the rate calculation
|
||||
val throttleTimeMs = difference / e.bound * windowSize(e.metric, timeMs)
|
||||
Math.round(throttleTimeMs)
|
||||
}
|
||||
|
||||
private def windowSize(metric: KafkaMetric, timeMs: Long): Long =
|
||||
measurableAsRate(metric.metricName, metric.measurable).windowSize(metric.config, timeMs)
|
||||
|
||||
// Casting to Rate because we only use Rate in Quota computation
|
||||
private def measurableAsRate(name: MetricName, measurable: Measurable): Rate = {
|
||||
measurable match {
|
||||
case r: Rate => r
|
||||
case _ => throw new IllegalArgumentException(s"Metric $name is not a Rate metric, value $measurable")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -155,31 +180,36 @@ object ClientQuotaManager {
|
|||
* @param metrics @Metrics Metrics instance
|
||||
* @param quotaType Quota type of this quota manager
|
||||
* @param time @Time object to use
|
||||
* @param threadNamePrefix The thread prefix to use
|
||||
* @param clientQuotaCallback An optional @ClientQuotaCallback
|
||||
*/
|
||||
class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
||||
private val metrics: Metrics,
|
||||
private val quotaType: QuotaType,
|
||||
private val time: Time,
|
||||
threadNamePrefix: String,
|
||||
clientQuotaCallback: Option[ClientQuotaCallback] = None) extends Logging {
|
||||
private val staticConfigClientIdQuota = Quota.upperBound(config.quotaBytesPerSecondDefault.toDouble)
|
||||
private val clientQuotaType = quotaTypeToClientQuotaType(quotaType)
|
||||
@volatile private var quotaTypesEnabled = clientQuotaCallback match {
|
||||
private val threadNamePrefix: String,
|
||||
private val clientQuotaCallback: Option[ClientQuotaCallback] = None) extends Logging {
|
||||
|
||||
private val lock = new ReentrantReadWriteLock()
|
||||
private val sensorAccessor = new SensorAccess(lock, metrics)
|
||||
private val quotaCallback = clientQuotaCallback.getOrElse(new DefaultQuotaCallback)
|
||||
private val staticConfigClientIdQuota = Quota.upperBound(config.quotaDefault.toDouble)
|
||||
private val clientQuotaType = QuotaType.toClientQuotaType(quotaType)
|
||||
|
||||
@volatile
|
||||
private var quotaTypesEnabled = clientQuotaCallback match {
|
||||
case Some(_) => QuotaTypes.CustomQuotas
|
||||
case None =>
|
||||
if (config.quotaBytesPerSecondDefault == Long.MaxValue) QuotaTypes.NoQuotas
|
||||
if (config.quotaDefault == Long.MaxValue) QuotaTypes.NoQuotas
|
||||
else QuotaTypes.ClientIdQuotaEnabled
|
||||
}
|
||||
private val lock = new ReentrantReadWriteLock()
|
||||
private val delayQueue = new DelayQueue[ThrottledChannel]()
|
||||
private val sensorAccessor = new SensorAccess(lock, metrics)
|
||||
private[server] val throttledChannelReaper = new ThrottledChannelReaper(delayQueue, threadNamePrefix)
|
||||
private val quotaCallback = clientQuotaCallback.getOrElse(new DefaultQuotaCallback)
|
||||
|
||||
private val delayQueueSensor = metrics.sensor(quotaType.toString + "-delayQueue")
|
||||
delayQueueSensor.add(metrics.metricName("queue-size",
|
||||
quotaType.toString,
|
||||
delayQueueSensor.add(metrics.metricName("queue-size", quotaType.toString,
|
||||
"Tracks the size of the delay queue"), new CumulativeSum())
|
||||
|
||||
private val delayQueue = new DelayQueue[ThrottledChannel]()
|
||||
private[server] val throttledChannelReaper = new ThrottledChannelReaper(delayQueue, threadNamePrefix)
|
||||
start() // Use start method to keep spotbugs happy
|
||||
private def start(): Unit = {
|
||||
throttledChannelReaper.start()
|
||||
|
@ -213,17 +243,15 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
def quotasEnabled: Boolean = quotaTypesEnabled != QuotaTypes.NoQuotas
|
||||
|
||||
/**
|
||||
* Records that a user/clientId changed produced/consumed bytes being throttled at the specified time. If quota has
|
||||
* been violated, return throttle time in milliseconds. Throttle time calculation may be overridden by sub-classes.
|
||||
* @param request client request
|
||||
* @param value amount of data in bytes or request processing time as a percentage
|
||||
* @param timeMs time to record the value at
|
||||
* @return throttle time in milliseconds
|
||||
*/
|
||||
* See {recordAndGetThrottleTimeMs}.
|
||||
*/
|
||||
def maybeRecordAndGetThrottleTimeMs(request: RequestChannel.Request, value: Double, timeMs: Long): Int = {
|
||||
maybeRecordAndGetThrottleTimeMs(request.session, request.header.clientId, value, timeMs)
|
||||
}
|
||||
|
||||
/**
|
||||
* See {recordAndGetThrottleTimeMs}.
|
||||
*/
|
||||
def maybeRecordAndGetThrottleTimeMs(session: Session, clientId: String, value: Double, timeMs: Long): Int = {
|
||||
// Record metrics only if quotas are enabled.
|
||||
if (quotasEnabled) {
|
||||
|
@ -234,7 +262,56 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns maximum value (produced/consume bytes or request processing time) that could be recorded without guaranteed throttling.
|
||||
* Records that a user/clientId accumulated or would like to accumulate the provided amount at the
|
||||
* the specified time, returns throttle time in milliseconds.
|
||||
*
|
||||
* @param session The session from which the user is extracted
|
||||
* @param clientId The client id
|
||||
* @param value The value to accumulate
|
||||
* @param timeMs The time at which to accumulate the value
|
||||
* @return The throttle time in milliseconds defines as the time to wait until the average
|
||||
* rate gets back to the defined quota
|
||||
*/
|
||||
def recordAndGetThrottleTimeMs(session: Session, clientId: String, value: Double, timeMs: Long): Int = {
|
||||
val clientSensors = getOrCreateQuotaSensors(session, clientId)
|
||||
try {
|
||||
clientSensors.quotaSensor.record(value, timeMs, true)
|
||||
0
|
||||
} catch {
|
||||
case e: QuotaViolationException =>
|
||||
val throttleTimeMs = throttleTime(e, timeMs).toInt
|
||||
debug(s"Quota violated for sensor (${clientSensors.quotaSensor.name}). Delay time: ($throttleTimeMs)")
|
||||
throttleTimeMs
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Records that a user/clientId changed some metric being throttled without checking for
|
||||
* quota violation. The aggregate value will subsequently be used for throttling when the
|
||||
* next request is processed.
|
||||
*/
|
||||
def recordNoThrottle(session: Session, clientId: String, value: Double): Unit = {
|
||||
val clientSensors = getOrCreateQuotaSensors(session, clientId)
|
||||
clientSensors.quotaSensor.record(value, time.milliseconds(), false)
|
||||
}
|
||||
|
||||
/**
|
||||
* "Unrecord" the given value that has already been recorded for the given user/client by recording a negative value
|
||||
* of the same quantity.
|
||||
*
|
||||
* For a throttled fetch, the broker should return an empty response and thus should not record the value. Ideally,
|
||||
* we would like to compute the throttle time before actually recording the value, but the current Sensor code
|
||||
* couples value recording and quota checking very tightly. As a workaround, we will unrecord the value for the fetch
|
||||
* in case of throttling. Rate keeps the sum of values that fall in each time window, so this should bring the
|
||||
* overall sum back to the previous value.
|
||||
*/
|
||||
def unrecordQuotaSensor(request: RequestChannel.Request, value: Double, timeMs: Long): Unit = {
|
||||
val clientSensors = getOrCreateQuotaSensors(request.session, request.header.clientId)
|
||||
clientSensors.quotaSensor.record(value * (-1), timeMs, false)
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns maximum value that could be recorded without guaranteed throttling.
|
||||
* Recording any larger value will always be throttled, even if no other values were recorded in the quota window.
|
||||
* This is used for deciding the maximum bytes that can be fetched at once
|
||||
*/
|
||||
|
@ -249,40 +326,13 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
}
|
||||
}
|
||||
|
||||
def recordAndGetThrottleTimeMs(session: Session, clientId: String, value: Double, timeMs: Long): Int = {
|
||||
val clientSensors = getOrCreateQuotaSensors(session, clientId)
|
||||
try {
|
||||
clientSensors.quotaSensor.record(value, timeMs)
|
||||
0
|
||||
} catch {
|
||||
case e: QuotaViolationException =>
|
||||
val throttleTimeMs = throttleTime(e.value, e.bound, windowSize(e.metric, timeMs)).toInt
|
||||
debug(s"Quota violated for sensor (${clientSensors.quotaSensor.name}). Delay time: ($throttleTimeMs)")
|
||||
throttleTimeMs
|
||||
}
|
||||
}
|
||||
|
||||
/** "Unrecord" the given value that has already been recorded for the given user/client by recording a negative value
|
||||
* of the same quantity.
|
||||
*
|
||||
* For a throttled fetch, the broker should return an empty response and thus should not record the value. Ideally,
|
||||
* we would like to compute the throttle time before actually recording the value, but the current Sensor code
|
||||
* couples value recording and quota checking very tightly. As a workaround, we will unrecord the value for the fetch
|
||||
* in case of throttling. Rate keeps the sum of values that fall in each time window, so this should bring the
|
||||
* overall sum back to the previous value.
|
||||
*/
|
||||
def unrecordQuotaSensor(request: RequestChannel.Request, value: Double, timeMs: Long): Unit = {
|
||||
val clientSensors = getOrCreateQuotaSensors(request.session, request.header.clientId)
|
||||
clientSensors.quotaSensor.record(value * (-1), timeMs, false)
|
||||
}
|
||||
|
||||
/**
|
||||
* Throttle a client by muting the associated channel for the given throttle time.
|
||||
* @param request client request
|
||||
* @param throttleTimeMs Duration in milliseconds for which the channel is to be muted.
|
||||
* @param channelThrottlingCallback Callback for channel throttling
|
||||
* @return ThrottledChannel object
|
||||
*/
|
||||
* Throttle a client by muting the associated channel for the given throttle time.
|
||||
*
|
||||
* @param request client request
|
||||
* @param throttleTimeMs Duration in milliseconds for which the channel is to be muted.
|
||||
* @param channelThrottlingCallback Callback for channel throttling
|
||||
*/
|
||||
def throttle(request: RequestChannel.Request, throttleTimeMs: Int, channelThrottlingCallback: Response => Unit): Unit = {
|
||||
if (throttleTimeMs > 0) {
|
||||
val clientSensors = getOrCreateQuotaSensors(request.session, request.header.clientId)
|
||||
|
@ -294,15 +344,6 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Records that a user/clientId changed some metric being throttled without checking for
|
||||
* quota violation. The aggregate value will subsequently be used for throttling when the
|
||||
* next request is processed.
|
||||
*/
|
||||
def recordNoThrottle(clientSensors: ClientSensors, value: Double): Unit = {
|
||||
clientSensors.quotaSensor.record(value, time.milliseconds(), false)
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the quota for the client with the specified (non-encoded) user principal and client-id.
|
||||
*
|
||||
|
@ -327,33 +368,17 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
Option(quotaCallback.quotaLimit(clientQuotaType, metricTags)).map(_.toDouble).getOrElse(Long.MaxValue)
|
||||
}
|
||||
|
||||
/*
|
||||
/**
|
||||
* This calculates the amount of time needed to bring the metric within quota
|
||||
* assuming that no new metrics are recorded.
|
||||
*
|
||||
* Basically, if O is the observed rate and T is the target rate over a window of W, to bring O down to T,
|
||||
* we need to add a delay of X to W such that O * W / (W + X) = T.
|
||||
* Solving for X, we get X = (O - T)/T * W.
|
||||
* See {ClientQuotaManager.throttleTime} for the details.
|
||||
*/
|
||||
protected def throttleTime(quotaValue: Double, quotaBound: Double, windowSize: Long): Long = {
|
||||
val difference = quotaValue - quotaBound
|
||||
// Use the precise window used by the rate calculation
|
||||
val throttleTimeMs = difference / quotaBound * windowSize
|
||||
Math.round(throttleTimeMs)
|
||||
protected def throttleTime(e: QuotaViolationException, timeMs: Long): Long = {
|
||||
ClientQuotaManager.throttleTime(e, timeMs)
|
||||
}
|
||||
|
||||
private def windowSize(metric: KafkaMetric, timeMs: Long): Long =
|
||||
measurableAsRate(metric.metricName, metric.measurable).windowSize(metric.config, timeMs)
|
||||
|
||||
// Casting to Rate because we only use Rate in Quota computation
|
||||
private def measurableAsRate(name: MetricName, measurable: Measurable): Rate = {
|
||||
measurable match {
|
||||
case r: Rate => r
|
||||
case _ => throw new IllegalArgumentException(s"Metric $name is not a Rate metric, value $measurable")
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
/**
|
||||
* This function either returns the sensors for a given client id or creates them if they don't exist
|
||||
* First sensor of the tuple is the quota enforcement sensor. Second one is the throttle time sensor
|
||||
*/
|
||||
|
@ -368,13 +393,14 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
metricTags,
|
||||
sensorAccessor.getOrCreate(
|
||||
getQuotaSensorName(metricTags),
|
||||
ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds,
|
||||
ClientQuotaManager.InactiveSensorExpirationTimeSeconds,
|
||||
clientRateMetricName(metricTags),
|
||||
Some(getQuotaMetricConfig(metricTags)),
|
||||
new Rate
|
||||
),
|
||||
sensorAccessor.getOrCreate(getThrottleTimeSensorName(metricTags),
|
||||
ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds,
|
||||
sensorAccessor.getOrCreate(
|
||||
getThrottleTimeSensorName(metricTags),
|
||||
ClientQuotaManager.InactiveSensorExpirationTimeSeconds,
|
||||
throttleMetricName(metricTags),
|
||||
None,
|
||||
new Avg
|
||||
|
@ -408,7 +434,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
protected def getOrCreateSensor(sensorName: String, metricName: MetricName): Sensor = {
|
||||
sensorAccessor.getOrCreate(
|
||||
sensorName,
|
||||
ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds,
|
||||
ClientQuotaManager.InactiveSensorExpirationTimeSeconds,
|
||||
metricName,
|
||||
None,
|
||||
new Rate
|
||||
|
@ -471,6 +497,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
* Updates metrics configs. This is invoked when quota configs are updated in ZooKeeper
|
||||
* or when partitions leaders change and custom callbacks that implement partition-based quotas
|
||||
* have updated quotas.
|
||||
*
|
||||
* @param updatedQuotaEntity If set to one entity and quotas have only been enabled at one
|
||||
* level, then an optimized update is performed with a single metric update. If None is provided,
|
||||
* or if custom callbacks are used or if multi-level quotas have been enabled, all metric configs
|
||||
|
@ -531,15 +558,6 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
quotaMetricTags.asJava)
|
||||
}
|
||||
|
||||
private def quotaTypeToClientQuotaType(quotaType: QuotaType): ClientQuotaType = {
|
||||
quotaType match {
|
||||
case QuotaType.Fetch => ClientQuotaType.FETCH
|
||||
case QuotaType.Produce => ClientQuotaType.PRODUCE
|
||||
case QuotaType.Request => ClientQuotaType.REQUEST
|
||||
case _ => throw new IllegalArgumentException(s"Not a client quota type: $quotaType")
|
||||
}
|
||||
}
|
||||
|
||||
def shutdown(): Unit = {
|
||||
throttledChannelReaper.shutdown()
|
||||
}
|
||||
|
|
|
@ -26,15 +26,25 @@ import org.apache.kafka.server.quota.ClientQuotaCallback
|
|||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
object ClientRequestQuotaManager {
|
||||
val QuotaRequestPercentDefault = Int.MaxValue.toDouble
|
||||
val NanosToPercentagePerSecond = 100.0 / TimeUnit.SECONDS.toNanos(1)
|
||||
|
||||
private val ExemptSensorName = "exempt-" + QuotaType.Request
|
||||
}
|
||||
|
||||
class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig,
|
||||
private val metrics: Metrics,
|
||||
private val time: Time,
|
||||
threadNamePrefix: String,
|
||||
quotaCallback: Option[ClientQuotaCallback])
|
||||
extends ClientQuotaManager(config, metrics, QuotaType.Request, time, threadNamePrefix, quotaCallback) {
|
||||
val maxThrottleTimeMs = TimeUnit.SECONDS.toMillis(this.config.quotaWindowSizeSeconds)
|
||||
def exemptSensor = getOrCreateSensor(exemptSensorName, exemptMetricName)
|
||||
private val threadNamePrefix: String,
|
||||
private val quotaCallback: Option[ClientQuotaCallback])
|
||||
extends ClientQuotaManager(config, metrics, QuotaType.Request, time, threadNamePrefix, quotaCallback) {
|
||||
|
||||
private val maxThrottleTimeMs = TimeUnit.SECONDS.toMillis(this.config.quotaWindowSizeSeconds)
|
||||
private val exemptMetricName = metrics.metricName("exempt-request-time",
|
||||
QuotaType.Request.toString, "Tracking exempt-request-time utilization percentage")
|
||||
|
||||
lazy val exemptSensor: Sensor = getOrCreateSensor(ClientRequestQuotaManager.ExemptSensorName, exemptMetricName)
|
||||
|
||||
def recordExempt(value: Double): Unit = {
|
||||
exemptSensor.record(value)
|
||||
|
@ -49,7 +59,7 @@ class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
def maybeRecordAndGetThrottleTimeMs(request: RequestChannel.Request, timeMs: Long): Int = {
|
||||
if (quotasEnabled) {
|
||||
request.recordNetworkThreadTimeCallback = Some(timeNanos => recordNoThrottle(
|
||||
getOrCreateQuotaSensors(request.session, request.header.clientId), nanosToPercentage(timeNanos)))
|
||||
request.session, request.header.clientId, nanosToPercentage(timeNanos)))
|
||||
recordAndGetThrottleTimeMs(request.session, request.header.clientId,
|
||||
nanosToPercentage(request.requestThreadTimeNanos), timeMs)
|
||||
} else {
|
||||
|
@ -64,8 +74,8 @@ class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
}
|
||||
}
|
||||
|
||||
override protected def throttleTime(quotaValue: Double, quotaBound: Double, windowSize: Long): Long = {
|
||||
math.min(super.throttleTime(quotaValue, quotaBound, windowSize), maxThrottleTimeMs)
|
||||
override protected def throttleTime(e: QuotaViolationException, timeMs: Long): Long = {
|
||||
math.min(super.throttleTime(e, timeMs), maxThrottleTimeMs)
|
||||
}
|
||||
|
||||
override protected def clientRateMetricName(quotaMetricTags: Map[String, String]): MetricName = {
|
||||
|
@ -74,13 +84,6 @@ class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig,
|
|||
quotaMetricTags.asJava)
|
||||
}
|
||||
|
||||
private def exemptMetricName: MetricName = {
|
||||
metrics.metricName("exempt-request-time", QuotaType.Request.toString,
|
||||
"Tracking exempt-request-time utilization percentage")
|
||||
}
|
||||
|
||||
private def exemptSensorName: String = "exempt-" + QuotaType.Request
|
||||
|
||||
private def nanosToPercentage(nanos: Long): Double = nanos * ClientQuotaManagerConfig.NanosToPercentagePerSecond
|
||||
|
||||
private def nanosToPercentage(nanos: Long): Double =
|
||||
nanos * ClientRequestQuotaManager.NanosToPercentagePerSecond
|
||||
}
|
||||
|
|
|
@ -144,6 +144,12 @@ class QuotaConfigHandler(private val quotaManagers: QuotaManagers) {
|
|||
else
|
||||
None
|
||||
quotaManagers.request.updateQuota(sanitizedUser, clientId, sanitizedClientId, requestQuota)
|
||||
val controllerMutationQuota =
|
||||
if (config.containsKey(DynamicConfig.Client.ControllerMutationOverrideProp))
|
||||
Some(new Quota(config.getProperty(DynamicConfig.Client.ControllerMutationOverrideProp).toLong.toDouble, true))
|
||||
else
|
||||
None
|
||||
quotaManagers.controllerMutation.updateQuota(sanitizedUser, clientId, sanitizedClientId, controllerMutationQuota)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,242 @@
|
|||
/**
|
||||
* 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 kafka.network.RequestChannel.Session
|
||||
import org.apache.kafka.common.MetricName
|
||||
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException
|
||||
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.protocol.Errors
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.server.quota.ClientQuotaCallback
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
/**
|
||||
* The ControllerMutationQuota trait defines a quota for a given user/clientId pair. Such
|
||||
* quota is not meant to be cached forever but rather during the lifetime of processing
|
||||
* a request.
|
||||
*/
|
||||
trait ControllerMutationQuota {
|
||||
def isExceeded: Boolean
|
||||
def record(permits: Double): Unit
|
||||
def throttleTime: Int
|
||||
}
|
||||
|
||||
/**
|
||||
* Default quota used when quota is disabled.
|
||||
*/
|
||||
object UnboundedControllerMutationQuota extends ControllerMutationQuota {
|
||||
override def isExceeded: Boolean = false
|
||||
override def record(permits: Double): Unit = ()
|
||||
override def throttleTime: Int = 0
|
||||
}
|
||||
|
||||
/**
|
||||
* The AbstractControllerMutationQuota is the base class of StrictControllerMutationQuota and
|
||||
* PermissiveControllerMutationQuota.
|
||||
*
|
||||
* @param time @Time object to use
|
||||
*/
|
||||
abstract class AbstractControllerMutationQuota(private val time: Time) extends ControllerMutationQuota {
|
||||
protected var lastThrottleTimeMs = 0L
|
||||
protected var lastRecordedTimeMs = 0L
|
||||
|
||||
protected def updateThrottleTime(e: QuotaViolationException, timeMs: Long): Unit = {
|
||||
lastThrottleTimeMs = ClientQuotaManager.throttleTime(e, timeMs)
|
||||
lastRecordedTimeMs = timeMs
|
||||
}
|
||||
|
||||
override def throttleTime: Int = {
|
||||
// If a throttle time has been recorded, we adjust it by deducting the time elapsed
|
||||
// between the recording and now. We do this because `throttleTime` may be called
|
||||
// long after having recorded it, especially when a request waits in the purgatory.
|
||||
val deltaTimeMs = time.milliseconds - lastRecordedTimeMs
|
||||
Math.max(0, lastThrottleTimeMs - deltaTimeMs).toInt
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The StrictControllerMutationQuota defines a strict quota for a given user/clientId pair. The
|
||||
* quota is strict meaning that 1) it does not accept any mutations once the quota is exhausted
|
||||
* until it gets back to the defined rate; and 2) it does not throttle for any number of mutations
|
||||
* if quota is not already exhausted.
|
||||
*
|
||||
* @param time @Time object to use
|
||||
* @param quotaSensor @Sensor object with a defined quota for a given user/clientId pair
|
||||
*/
|
||||
class StrictControllerMutationQuota(private val time: Time,
|
||||
private val quotaSensor: Sensor)
|
||||
extends AbstractControllerMutationQuota(time) {
|
||||
|
||||
override def isExceeded: Boolean = lastThrottleTimeMs > 0
|
||||
|
||||
override def record(permits: Double): Unit = {
|
||||
val timeMs = time.milliseconds
|
||||
try {
|
||||
quotaSensor synchronized {
|
||||
quotaSensor.checkQuotas(timeMs)
|
||||
quotaSensor.record(permits, timeMs, false)
|
||||
}
|
||||
} catch {
|
||||
case e: QuotaViolationException =>
|
||||
updateThrottleTime(e, timeMs)
|
||||
throw new ThrottlingQuotaExceededException(lastThrottleTimeMs.toInt,
|
||||
Errors.THROTTLING_QUOTA_EXCEEDED.message)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The PermissiveControllerMutationQuota defines a permissive quota for a given user/clientId pair.
|
||||
* The quota is permissive meaning that 1) it does accept any mutations even if the quota is
|
||||
* exhausted; and 2) it does throttle as soon as the quota is exhausted.
|
||||
*
|
||||
* @param time @Time object to use
|
||||
* @param quotaSensor @Sensor object with a defined quota for a given user/clientId pair
|
||||
*/
|
||||
class PermissiveControllerMutationQuota(private val time: Time,
|
||||
private val quotaSensor: Sensor)
|
||||
extends AbstractControllerMutationQuota(time) {
|
||||
|
||||
override def isExceeded: Boolean = false
|
||||
|
||||
override def record(permits: Double): Unit = {
|
||||
val timeMs = time.milliseconds
|
||||
try {
|
||||
quotaSensor.record(permits, timeMs, true)
|
||||
} catch {
|
||||
case e: QuotaViolationException =>
|
||||
updateThrottleTime(e, timeMs)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The ControllerMutationQuotaManager is a specialized ClientQuotaManager used in the context
|
||||
* of throttling controller's operations/mutations.
|
||||
*
|
||||
* @param config @ClientQuotaManagerConfig quota configs
|
||||
* @param metrics @Metrics Metrics instance
|
||||
* @param time @Time object to use
|
||||
* @param threadNamePrefix The thread prefix to use
|
||||
* @param quotaCallback @ClientQuotaCallback ClientQuotaCallback to use
|
||||
*/
|
||||
class ControllerMutationQuotaManager(private val config: ClientQuotaManagerConfig,
|
||||
private val metrics: Metrics,
|
||||
private val time: Time,
|
||||
private val threadNamePrefix: String,
|
||||
private val quotaCallback: Option[ClientQuotaCallback])
|
||||
extends ClientQuotaManager(config, metrics, QuotaType.ControllerMutation, time, threadNamePrefix, quotaCallback) {
|
||||
|
||||
override protected def clientRateMetricName(quotaMetricTags: Map[String, String]): MetricName = {
|
||||
metrics.metricName("mutation-rate", QuotaType.ControllerMutation.toString,
|
||||
"Tracking mutation-rate per user/client-id",
|
||||
quotaMetricTags.asJava)
|
||||
}
|
||||
|
||||
/**
|
||||
* Records that a user/clientId accumulated or would like to accumulate the provided amount at the
|
||||
* the specified time, returns throttle time in milliseconds. The quota is strict meaning that it
|
||||
* does not accept any mutations once the quota is exhausted until it gets back to the defined rate.
|
||||
*
|
||||
* @param session The session from which the user is extracted
|
||||
* @param clientId The client id
|
||||
* @param value The value to accumulate
|
||||
* @param timeMs The time at which to accumulate the value
|
||||
* @return The throttle time in milliseconds defines as the time to wait until the average
|
||||
* rate gets back to the defined quota
|
||||
*/
|
||||
override def recordAndGetThrottleTimeMs(session: Session, clientId: String, value: Double, timeMs: Long): Int = {
|
||||
val clientSensors = getOrCreateQuotaSensors(session, clientId)
|
||||
val quotaSensor = clientSensors.quotaSensor
|
||||
try {
|
||||
quotaSensor synchronized {
|
||||
quotaSensor.checkQuotas(timeMs)
|
||||
quotaSensor.record(value, timeMs, false)
|
||||
}
|
||||
0
|
||||
} catch {
|
||||
case e: QuotaViolationException =>
|
||||
val throttleTimeMs = throttleTime(e, timeMs).toInt
|
||||
debug(s"Quota violated for sensor (${quotaSensor.name}). Delay time: ($throttleTimeMs)")
|
||||
throttleTimeMs
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a StrictControllerMutationQuota for the given user/clientId pair or
|
||||
* a UnboundedControllerMutationQuota$ if the quota is disabled.
|
||||
*
|
||||
* @param session The session from which the user is extracted
|
||||
* @param clientId The client id
|
||||
* @return ControllerMutationQuota
|
||||
*/
|
||||
def newStrictQuotaFor(session: Session, clientId: String): ControllerMutationQuota = {
|
||||
if (quotasEnabled) {
|
||||
val clientSensors = getOrCreateQuotaSensors(session, clientId)
|
||||
new StrictControllerMutationQuota(time, clientSensors.quotaSensor)
|
||||
} else {
|
||||
UnboundedControllerMutationQuota
|
||||
}
|
||||
}
|
||||
|
||||
def newStrictQuotaFor(request: RequestChannel.Request): ControllerMutationQuota =
|
||||
newStrictQuotaFor(request.session, request.header.clientId)
|
||||
|
||||
/**
|
||||
* Returns a PermissiveControllerMutationQuota for the given user/clientId pair or
|
||||
* a UnboundedControllerMutationQuota$ if the quota is disabled.
|
||||
*
|
||||
* @param session The session from which the user is extracted
|
||||
* @param clientId The client id
|
||||
* @return ControllerMutationQuota
|
||||
*/
|
||||
def newPermissiveQuotaFor(session: Session, clientId: String): ControllerMutationQuota = {
|
||||
if (quotasEnabled) {
|
||||
val clientSensors = getOrCreateQuotaSensors(session, clientId)
|
||||
new PermissiveControllerMutationQuota(time, clientSensors.quotaSensor)
|
||||
} else {
|
||||
UnboundedControllerMutationQuota
|
||||
}
|
||||
}
|
||||
|
||||
def newPermissiveQuotaFor(request: RequestChannel.Request): ControllerMutationQuota =
|
||||
newPermissiveQuotaFor(request.session, request.header.clientId)
|
||||
|
||||
/**
|
||||
* Returns a ControllerMutationQuota based on `strictSinceVersion`. It returns a strict
|
||||
* quota if the version is equal to or above of the `strictSinceVersion`, a permissive
|
||||
* quota if the version is below, and a unbounded quota if the quota is disabled.
|
||||
*
|
||||
* When the quota is strictly enforced. Any operation above the quota is not allowed
|
||||
* and rejected with a THROTTLING_QUOTA_EXCEEDED error.
|
||||
*
|
||||
* @param request The request to extract the user and the clientId from
|
||||
* @param strictSinceVersion The version since quota is strict
|
||||
* @return
|
||||
*/
|
||||
def newQuotaFor(request: RequestChannel.Request, strictSinceVersion: Short): ControllerMutationQuota = {
|
||||
if (request.header.apiVersion() >= strictSinceVersion)
|
||||
newStrictQuotaFor(request)
|
||||
else
|
||||
newPermissiveQuotaFor(request)
|
||||
}
|
||||
}
|
|
@ -28,6 +28,20 @@ import scala.collection._
|
|||
*/
|
||||
case class CreatePartitionsMetadata(topic: String, partitions: Set[Int], error: ApiError)
|
||||
|
||||
object CreatePartitionsMetadata {
|
||||
def apply(topic: String, partitions: Set[Int]): CreatePartitionsMetadata = {
|
||||
CreatePartitionsMetadata(topic, partitions, ApiError.NONE)
|
||||
}
|
||||
|
||||
def apply(topic: String, error: Errors): CreatePartitionsMetadata = {
|
||||
CreatePartitionsMetadata(topic, Set.empty, new ApiError(error, null))
|
||||
}
|
||||
|
||||
def apply(topic: String, throwable: Throwable): CreatePartitionsMetadata = {
|
||||
CreatePartitionsMetadata(topic, Set.empty, ApiError.fromThrowable(throwable))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A delayed create topic or create partitions operation that is stored in the topic purgatory.
|
||||
*/
|
||||
|
|
|
@ -26,6 +26,12 @@ import scala.collection._
|
|||
*/
|
||||
case class DeleteTopicMetadata(topic: String, error: Errors)
|
||||
|
||||
object DeleteTopicMetadata {
|
||||
def apply(topic: String, throwable: Throwable): DeleteTopicMetadata = {
|
||||
DeleteTopicMetadata(topic, Errors.forException(throwable))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A delayed delete topics operation that can be created by the admin manager and watched
|
||||
* in the topic purgatory
|
||||
|
|
|
@ -35,15 +35,15 @@ import scala.jdk.CollectionConverters._
|
|||
object DynamicConfig {
|
||||
|
||||
object Broker {
|
||||
//Properties
|
||||
// Properties
|
||||
val LeaderReplicationThrottledRateProp = "leader.replication.throttled.rate"
|
||||
val FollowerReplicationThrottledRateProp = "follower.replication.throttled.rate"
|
||||
val ReplicaAlterLogDirsIoMaxBytesPerSecondProp = "replica.alter.log.dirs.io.max.bytes.per.second"
|
||||
|
||||
//Defaults
|
||||
// Defaults
|
||||
val DefaultReplicationThrottledRate = ReplicationQuotaManagerConfig.QuotaBytesPerSecondDefault
|
||||
|
||||
//Documentation
|
||||
// Documentation
|
||||
val LeaderReplicationThrottledRateDoc = "A long representing the upper bound (bytes/sec) on replication traffic for leaders enumerated in the " +
|
||||
s"property ${LogConfig.LeaderReplicationThrottledReplicasProp} (for each topic). This property can be only set dynamically. It is suggested that the " +
|
||||
s"limit be kept above 1MB/s for accurate behaviour."
|
||||
|
@ -53,9 +53,9 @@ object DynamicConfig {
|
|||
val ReplicaAlterLogDirsIoMaxBytesPerSecondDoc = "A long representing the upper bound (bytes/sec) on disk IO used for moving replica between log directories on the same broker. " +
|
||||
s"This property can be only set dynamically. It is suggested that the limit be kept above 1MB/s for accurate behaviour."
|
||||
|
||||
//Definitions
|
||||
// Definitions
|
||||
val brokerConfigDef = new ConfigDef()
|
||||
//round minimum value down, to make it easier for users.
|
||||
// Round minimum value down, to make it easier for users.
|
||||
.define(LeaderReplicationThrottledRateProp, LONG, DefaultReplicationThrottledRate, atLeast(0), MEDIUM, LeaderReplicationThrottledRateDoc)
|
||||
.define(FollowerReplicationThrottledRateProp, LONG, DefaultReplicationThrottledRate, atLeast(0), MEDIUM, FollowerReplicationThrottledRateDoc)
|
||||
.define(ReplicaAlterLogDirsIoMaxBytesPerSecondProp, LONG, DefaultReplicationThrottledRate, atLeast(0), MEDIUM, ReplicaAlterLogDirsIoMaxBytesPerSecondDoc)
|
||||
|
@ -71,32 +71,39 @@ object DynamicConfig {
|
|||
val ProducerByteRateOverrideProp = "producer_byte_rate"
|
||||
val ConsumerByteRateOverrideProp = "consumer_byte_rate"
|
||||
val RequestPercentageOverrideProp = "request_percentage"
|
||||
private val configNames = Set(ProducerByteRateOverrideProp, ConsumerByteRateOverrideProp, RequestPercentageOverrideProp)
|
||||
val ControllerMutationOverrideProp = "controller_mutation_rate"
|
||||
private val configNames = Set(ProducerByteRateOverrideProp, ConsumerByteRateOverrideProp,
|
||||
RequestPercentageOverrideProp, ControllerMutationOverrideProp)
|
||||
|
||||
def isQuotaConfig(name: String): Boolean = configNames.contains(name)
|
||||
}
|
||||
|
||||
object Client {
|
||||
//Properties
|
||||
// Properties
|
||||
val ProducerByteRateOverrideProp = QuotaConfigs.ProducerByteRateOverrideProp
|
||||
val ConsumerByteRateOverrideProp = QuotaConfigs.ConsumerByteRateOverrideProp
|
||||
val RequestPercentageOverrideProp = QuotaConfigs.RequestPercentageOverrideProp
|
||||
val ControllerMutationOverrideProp = QuotaConfigs.ControllerMutationOverrideProp
|
||||
|
||||
//Defaults
|
||||
val DefaultProducerOverride = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault
|
||||
val DefaultConsumerOverride = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault
|
||||
val DefaultRequestOverride = ClientQuotaManagerConfig.QuotaRequestPercentDefault
|
||||
// Defaults
|
||||
val DefaultProducerOverride = ClientQuotaManagerConfig.QuotaDefault
|
||||
val DefaultConsumerOverride = ClientQuotaManagerConfig.QuotaDefault
|
||||
val DefaultRequestOverride = ClientRequestQuotaManager.QuotaRequestPercentDefault
|
||||
val DefaultControllerMutationOverride = ClientQuotaManagerConfig.QuotaDefault
|
||||
|
||||
//Documentation
|
||||
// Documentation
|
||||
val ProducerOverrideDoc = "A rate representing the upper bound (bytes/sec) for producer traffic."
|
||||
val ConsumerOverrideDoc = "A rate representing the upper bound (bytes/sec) for consumer traffic."
|
||||
val RequestOverrideDoc = "A percentage representing the upper bound of time spent for processing requests."
|
||||
val ControllerMutationOverrideDoc = "The rate at which mutations are accepted for the create topics request, " +
|
||||
"the create partitions request and the delete topics request. The rate is accumulated by the number of partitions created or deleted."
|
||||
|
||||
//Definitions
|
||||
// Definitions
|
||||
private val clientConfigs = new ConfigDef()
|
||||
.define(ProducerByteRateOverrideProp, LONG, DefaultProducerOverride, MEDIUM, ProducerOverrideDoc)
|
||||
.define(ConsumerByteRateOverrideProp, LONG, DefaultConsumerOverride, MEDIUM, ConsumerOverrideDoc)
|
||||
.define(RequestPercentageOverrideProp, DOUBLE, DefaultRequestOverride, MEDIUM, RequestOverrideDoc)
|
||||
.define(ControllerMutationOverrideProp, LONG, DefaultConsumerOverride, MEDIUM, ControllerMutationOverrideDoc)
|
||||
|
||||
def configKeys = clientConfigs.configKeys
|
||||
|
||||
|
@ -106,12 +113,12 @@ object DynamicConfig {
|
|||
}
|
||||
|
||||
object User {
|
||||
|
||||
//Definitions
|
||||
// Definitions
|
||||
private val userConfigs = CredentialProvider.userCredentialConfigs
|
||||
.define(Client.ProducerByteRateOverrideProp, LONG, Client.DefaultProducerOverride, MEDIUM, Client.ProducerOverrideDoc)
|
||||
.define(Client.ConsumerByteRateOverrideProp, LONG, Client.DefaultConsumerOverride, MEDIUM, Client.ConsumerOverrideDoc)
|
||||
.define(Client.RequestPercentageOverrideProp, DOUBLE, Client.DefaultRequestOverride, MEDIUM, Client.RequestOverrideDoc)
|
||||
.define(Client.ControllerMutationOverrideProp, LONG, Client.DefaultConsumerOverride, MEDIUM, Client.ControllerMutationOverrideDoc)
|
||||
|
||||
def configKeys = userConfigs.configKeys
|
||||
|
||||
|
@ -121,7 +128,7 @@ object DynamicConfig {
|
|||
}
|
||||
|
||||
private def validate(configDef: ConfigDef, props: Properties, customPropsAllowed: Boolean) = {
|
||||
//Validate Names
|
||||
// Validate Names
|
||||
val names = configDef.names()
|
||||
val propKeys = props.keySet.asScala.map(_.asInstanceOf[String])
|
||||
if (!customPropsAllowed) {
|
||||
|
@ -129,7 +136,7 @@ object DynamicConfig {
|
|||
require(unknownKeys.isEmpty, s"Unknown Dynamic Configuration: $unknownKeys.")
|
||||
}
|
||||
val propResolved = DynamicBrokerConfig.resolveVariableConfigs(props)
|
||||
//ValidateValues
|
||||
// ValidateValues
|
||||
configDef.parse(propResolved)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -312,6 +312,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
quotas.fetch.updateQuotaMetricConfigs()
|
||||
quotas.produce.updateQuotaMetricConfigs()
|
||||
quotas.request.updateQuotaMetricConfigs()
|
||||
quotas.controllerMutation.updateQuotaMetricConfigs()
|
||||
}
|
||||
}
|
||||
if (replicaManager.hasDelayedElectionOperations) {
|
||||
|
@ -1684,25 +1685,27 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
}
|
||||
|
||||
def handleCreateTopicsRequest(request: RequestChannel.Request): Unit = {
|
||||
val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 6)
|
||||
|
||||
def sendResponseCallback(results: CreatableTopicResultCollection): Unit = {
|
||||
def createResponse(requestThrottleMs: Int): AbstractResponse = {
|
||||
val responseData = new CreateTopicsResponseData().
|
||||
setThrottleTimeMs(requestThrottleMs).
|
||||
setTopics(results)
|
||||
val responseData = new CreateTopicsResponseData()
|
||||
.setThrottleTimeMs(requestThrottleMs)
|
||||
.setTopics(results)
|
||||
val responseBody = new CreateTopicsResponse(responseData)
|
||||
trace(s"Sending create topics response $responseData for correlation id " +
|
||||
s"${request.header.correlationId} to client ${request.header.clientId}.")
|
||||
responseBody
|
||||
}
|
||||
sendResponseMaybeThrottle(request, createResponse)
|
||||
sendResponseMaybeThrottle(controllerMutationQuota, request, createResponse, onComplete = None)
|
||||
}
|
||||
|
||||
val createTopicsRequest = request.body[CreateTopicsRequest]
|
||||
val results = new CreatableTopicResultCollection(createTopicsRequest.data.topics.size)
|
||||
if (!controller.isActive) {
|
||||
createTopicsRequest.data.topics.forEach { topic =>
|
||||
results.add(new CreatableTopicResult().setName(topic.name).
|
||||
setErrorCode(Errors.NOT_CONTROLLER.code))
|
||||
results.add(new CreatableTopicResult().setName(topic.name)
|
||||
.setErrorCode(Errors.NOT_CONTROLLER.code))
|
||||
}
|
||||
sendResponseCallback(results)
|
||||
} else {
|
||||
|
@ -1718,7 +1721,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
val authorizedForDescribeConfigs = filterByAuthorized(request.context, DESCRIBE_CONFIGS, TOPIC,
|
||||
topics, logIfDenied = false)(identity).map(name => name -> results.find(name)).toMap
|
||||
|
||||
results.forEach(topic => {
|
||||
results.forEach { topic =>
|
||||
if (results.findAll(topic.name).size > 1) {
|
||||
topic.setErrorCode(Errors.INVALID_REQUEST.code)
|
||||
topic.setErrorMessage("Found multiple entries for this topic.")
|
||||
|
@ -1729,7 +1732,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
if (!authorizedForDescribeConfigs.contains(topic.name)) {
|
||||
topic.setTopicConfigErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)
|
||||
}
|
||||
})
|
||||
}
|
||||
val toCreate = mutable.Map[String, CreatableTopic]()
|
||||
createTopicsRequest.data.topics.forEach { topic =>
|
||||
if (results.find(topic.name).errorCode == Errors.NONE.code) {
|
||||
|
@ -1746,21 +1749,24 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
result.setConfigs(List.empty.asJava)
|
||||
.setNumPartitions(-1)
|
||||
.setReplicationFactor(-1)
|
||||
.setTopicConfigErrorCode(0.toShort)
|
||||
.setTopicConfigErrorCode(Errors.NONE.code)
|
||||
}
|
||||
}
|
||||
sendResponseCallback(results)
|
||||
}
|
||||
adminManager.createTopics(createTopicsRequest.data.timeoutMs,
|
||||
createTopicsRequest.data.validateOnly,
|
||||
toCreate,
|
||||
authorizedForDescribeConfigs,
|
||||
handleCreateTopicsResults)
|
||||
adminManager.createTopics(
|
||||
createTopicsRequest.data.timeoutMs,
|
||||
createTopicsRequest.data.validateOnly,
|
||||
toCreate,
|
||||
authorizedForDescribeConfigs,
|
||||
controllerMutationQuota,
|
||||
handleCreateTopicsResults)
|
||||
}
|
||||
}
|
||||
|
||||
def handleCreatePartitionsRequest(request: RequestChannel.Request): Unit = {
|
||||
val createPartitionsRequest = request.body[CreatePartitionsRequest]
|
||||
val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 3)
|
||||
|
||||
def sendResponseCallback(results: Map[String, ApiError]): Unit = {
|
||||
def createResponse(requestThrottleMs: Int): AbstractResponse = {
|
||||
|
@ -1777,7 +1783,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
s"client ${request.header.clientId}.")
|
||||
responseBody
|
||||
}
|
||||
sendResponseMaybeThrottle(request, createResponse)
|
||||
sendResponseMaybeThrottle(controllerMutationQuota, request, createResponse, onComplete = None)
|
||||
}
|
||||
|
||||
if (!controller.isActive) {
|
||||
|
@ -1803,14 +1809,18 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
unauthorized.map(_.name -> new ApiError(Errors.TOPIC_AUTHORIZATION_FAILED, "The topic authorization is failed.")) ++
|
||||
queuedForDeletion.map(_.name -> new ApiError(Errors.INVALID_TOPIC_EXCEPTION, "The topic is queued for deletion."))
|
||||
|
||||
adminManager.createPartitions(createPartitionsRequest.data.timeoutMs,
|
||||
adminManager.createPartitions(
|
||||
createPartitionsRequest.data.timeoutMs,
|
||||
valid,
|
||||
createPartitionsRequest.data.validateOnly,
|
||||
request.context.listenerName, result => sendResponseCallback(result ++ errors))
|
||||
controllerMutationQuota,
|
||||
result => sendResponseCallback(result ++ errors))
|
||||
}
|
||||
}
|
||||
|
||||
def handleDeleteTopicsRequest(request: RequestChannel.Request): Unit = {
|
||||
val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 5)
|
||||
|
||||
def sendResponseCallback(results: DeletableTopicResultCollection): Unit = {
|
||||
def createResponse(requestThrottleMs: Int): AbstractResponse = {
|
||||
val responseData = new DeleteTopicsResponseData()
|
||||
|
@ -1820,7 +1830,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
trace(s"Sending delete topics response $responseBody for correlation id ${request.header.correlationId} to client ${request.header.clientId}.")
|
||||
responseBody
|
||||
}
|
||||
sendResponseMaybeThrottle(request, createResponse)
|
||||
sendResponseMaybeThrottle(controllerMutationQuota, request, createResponse, onComplete = None)
|
||||
}
|
||||
|
||||
val deleteTopicRequest = request.body[DeleteTopicsRequest]
|
||||
|
@ -1872,6 +1882,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
adminManager.deleteTopics(
|
||||
deleteTopicRequest.data.timeoutMs,
|
||||
toDelete,
|
||||
controllerMutationQuota,
|
||||
handleDeleteTopicsResults
|
||||
)
|
||||
}
|
||||
|
@ -3090,6 +3101,30 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
throttleTimeMs
|
||||
}
|
||||
|
||||
/**
|
||||
* Throttle the channel if the controller mutations quota or the request quota have been violated.
|
||||
* Regardless of throttling, send the response immediately.
|
||||
*/
|
||||
private def sendResponseMaybeThrottle(controllerMutationQuota: ControllerMutationQuota,
|
||||
request: RequestChannel.Request,
|
||||
createResponse: Int => AbstractResponse,
|
||||
onComplete: Option[Send => Unit]): Unit = {
|
||||
val timeMs = time.milliseconds
|
||||
val controllerThrottleTimeMs = controllerMutationQuota.throttleTime
|
||||
val requestThrottleTimeMs = quotas.request.maybeRecordAndGetThrottleTimeMs(request, timeMs)
|
||||
val maxThrottleTimeMs = Math.max(controllerThrottleTimeMs, requestThrottleTimeMs)
|
||||
if (maxThrottleTimeMs > 0) {
|
||||
request.apiThrottleTimeMs = maxThrottleTimeMs
|
||||
if (controllerThrottleTimeMs > requestThrottleTimeMs) {
|
||||
quotas.controllerMutation.throttle(request, controllerThrottleTimeMs, requestChannel.sendResponse)
|
||||
} else {
|
||||
quotas.request.throttle(request, requestThrottleTimeMs, requestChannel.sendResponse)
|
||||
}
|
||||
}
|
||||
|
||||
sendResponse(request, Some(createResponse(maxThrottleTimeMs)), onComplete)
|
||||
}
|
||||
|
||||
private def sendResponseExemptThrottle(request: RequestChannel.Request,
|
||||
response: AbstractResponse,
|
||||
onComplete: Option[Send => Unit] = None): Unit = {
|
||||
|
|
|
@ -196,14 +196,16 @@ object Defaults {
|
|||
val FetchMaxBytes = 55 * 1024 * 1024
|
||||
|
||||
/** ********* Quota Configuration ***********/
|
||||
val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault
|
||||
val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault
|
||||
val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaDefault
|
||||
val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaDefault
|
||||
val NumQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples
|
||||
val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds
|
||||
val NumReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DefaultNumQuotaSamples
|
||||
val ReplicationQuotaWindowSizeSeconds: Int = ReplicationQuotaManagerConfig.DefaultQuotaWindowSizeSeconds
|
||||
val NumAlterLogDirsReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DefaultNumQuotaSamples
|
||||
val AlterLogDirsReplicationQuotaWindowSizeSeconds: Int = ReplicationQuotaManagerConfig.DefaultQuotaWindowSizeSeconds
|
||||
val NumControllerQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples
|
||||
val ControllerQuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds
|
||||
|
||||
/** ********* Transaction Configuration ***********/
|
||||
val TransactionalIdExpirationMsDefault = 604800000
|
||||
|
@ -483,9 +485,11 @@ object KafkaConfig {
|
|||
val NumQuotaSamplesProp = "quota.window.num"
|
||||
val NumReplicationQuotaSamplesProp = "replication.quota.window.num"
|
||||
val NumAlterLogDirsReplicationQuotaSamplesProp = "alter.log.dirs.replication.quota.window.num"
|
||||
val NumControllerQuotaSamplesProp = "controller.quota.window.num"
|
||||
val QuotaWindowSizeSecondsProp = "quota.window.size.seconds"
|
||||
val ReplicationQuotaWindowSizeSecondsProp = "replication.quota.window.size.seconds"
|
||||
val AlterLogDirsReplicationQuotaWindowSizeSecondsProp = "alter.log.dirs.replication.quota.window.size.seconds"
|
||||
val ControllerQuotaWindowSizeSecondsProp = "controller.quota.window.size.seconds"
|
||||
val ClientQuotaCallbackClassProp = "client.quota.callback.class"
|
||||
|
||||
val DeleteTopicEnableProp = "delete.topic.enable"
|
||||
|
@ -864,9 +868,12 @@ object KafkaConfig {
|
|||
val NumQuotaSamplesDoc = "The number of samples to retain in memory for client quotas"
|
||||
val NumReplicationQuotaSamplesDoc = "The number of samples to retain in memory for replication quotas"
|
||||
val NumAlterLogDirsReplicationQuotaSamplesDoc = "The number of samples to retain in memory for alter log dirs replication quotas"
|
||||
val NumControllerQuotaSamplesDoc = "The number of samples to retain in memory for controller mutation quotas"
|
||||
val QuotaWindowSizeSecondsDoc = "The time span of each sample for client quotas"
|
||||
val ReplicationQuotaWindowSizeSecondsDoc = "The time span of each sample for replication quotas"
|
||||
val AlterLogDirsReplicationQuotaWindowSizeSecondsDoc = "The time span of each sample for alter log dirs replication quotas"
|
||||
val ControllerQuotaWindowSizeSecondsDoc = "The time span of each sample for controller mutations quotas"
|
||||
|
||||
val ClientQuotaCallbackClassDoc = "The fully qualified name of a class that implements the ClientQuotaCallback interface, " +
|
||||
"which is used to determine quota limits applied to client requests. By default, <user, client-id>, <user> or <client-id> " +
|
||||
"quotas stored in ZooKeeper are applied. For any given request, the most specific quota that matches the user principal " +
|
||||
|
@ -1155,9 +1162,11 @@ object KafkaConfig {
|
|||
.define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc)
|
||||
.define(NumReplicationQuotaSamplesProp, INT, Defaults.NumReplicationQuotaSamples, atLeast(1), LOW, NumReplicationQuotaSamplesDoc)
|
||||
.define(NumAlterLogDirsReplicationQuotaSamplesProp, INT, Defaults.NumAlterLogDirsReplicationQuotaSamples, atLeast(1), LOW, NumAlterLogDirsReplicationQuotaSamplesDoc)
|
||||
.define(NumControllerQuotaSamplesProp, INT, Defaults.NumControllerQuotaSamples, atLeast(1), LOW, NumControllerQuotaSamplesDoc)
|
||||
.define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc)
|
||||
.define(ReplicationQuotaWindowSizeSecondsProp, INT, Defaults.ReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, ReplicationQuotaWindowSizeSecondsDoc)
|
||||
.define(AlterLogDirsReplicationQuotaWindowSizeSecondsProp, INT, Defaults.AlterLogDirsReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, AlterLogDirsReplicationQuotaWindowSizeSecondsDoc)
|
||||
.define(ControllerQuotaWindowSizeSecondsProp, INT, Defaults.ControllerQuotaWindowSizeSeconds, atLeast(1), LOW, ControllerQuotaWindowSizeSecondsDoc)
|
||||
.define(ClientQuotaCallbackClassProp, CLASS, null, LOW, ClientQuotaCallbackClassDoc)
|
||||
|
||||
/** ********* General Security Configuration ****************/
|
||||
|
@ -1600,6 +1609,8 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO
|
|||
val replicationQuotaWindowSizeSeconds = getInt(KafkaConfig.ReplicationQuotaWindowSizeSecondsProp)
|
||||
val numAlterLogDirsReplicationQuotaSamples = getInt(KafkaConfig.NumAlterLogDirsReplicationQuotaSamplesProp)
|
||||
val alterLogDirsReplicationQuotaWindowSizeSeconds = getInt(KafkaConfig.AlterLogDirsReplicationQuotaWindowSizeSecondsProp)
|
||||
val numControllerQuotaSamples = getInt(KafkaConfig.NumControllerQuotaSamplesProp)
|
||||
val controllerQuotaWindowSizeSeconds = getInt(KafkaConfig.ControllerQuotaWindowSizeSecondsProp)
|
||||
|
||||
/** ********* Fetch Configuration **************/
|
||||
val maxIncrementalFetchSessionCacheSlots = getInt(KafkaConfig.MaxIncrementalFetchSessionCacheSlots)
|
||||
|
|
|
@ -222,6 +222,10 @@ class MetadataCache(brokerId: Int) extends Logging {
|
|||
metadataSnapshot.partitionStates.get(topic).flatMap(_.get(partitionId))
|
||||
}
|
||||
|
||||
def numPartitions(topic: String): Option[Int] = {
|
||||
metadataSnapshot.partitionStates.get(topic).map(_.size)
|
||||
}
|
||||
|
||||
// if the leader is not known, return None;
|
||||
// if the leader is known and corresponding node is available, return Some(node)
|
||||
// if the leader is known but corresponding node with the listener name is not available, return Some(NO_NODE)
|
||||
|
|
|
@ -22,15 +22,28 @@ import org.apache.kafka.common.TopicPartition
|
|||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.server.quota.ClientQuotaCallback
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.kafka.server.quota.ClientQuotaType
|
||||
|
||||
object QuotaType {
|
||||
case object Fetch extends QuotaType
|
||||
case object Produce extends QuotaType
|
||||
case object Request extends QuotaType
|
||||
case object ControllerMutation extends QuotaType
|
||||
case object LeaderReplication extends QuotaType
|
||||
case object FollowerReplication extends QuotaType
|
||||
case object AlterLogDirsReplication extends QuotaType
|
||||
|
||||
def toClientQuotaType(quotaType: QuotaType): ClientQuotaType = {
|
||||
quotaType match {
|
||||
case QuotaType.Fetch => ClientQuotaType.FETCH
|
||||
case QuotaType.Produce => ClientQuotaType.PRODUCE
|
||||
case QuotaType.Request => ClientQuotaType.REQUEST
|
||||
case QuotaType.ControllerMutation => ClientQuotaType.CONTROLLER_MUTATION
|
||||
case _ => throw new IllegalArgumentException(s"Not a client quota type: $quotaType")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
sealed trait QuotaType
|
||||
|
||||
object QuotaFactory extends Logging {
|
||||
|
@ -44,6 +57,7 @@ object QuotaFactory extends Logging {
|
|||
case class QuotaManagers(fetch: ClientQuotaManager,
|
||||
produce: ClientQuotaManager,
|
||||
request: ClientRequestQuotaManager,
|
||||
controllerMutation: ControllerMutationQuotaManager,
|
||||
leader: ReplicationQuotaManager,
|
||||
follower: ReplicationQuotaManager,
|
||||
alterLogDirs: ReplicationQuotaManager,
|
||||
|
@ -52,6 +66,7 @@ object QuotaFactory extends Logging {
|
|||
fetch.shutdown()
|
||||
produce.shutdown()
|
||||
request.shutdown()
|
||||
controllerMutation.shutdown()
|
||||
clientQuotaCallback.foreach(_.close())
|
||||
}
|
||||
}
|
||||
|
@ -64,6 +79,8 @@ object QuotaFactory extends Logging {
|
|||
new ClientQuotaManager(clientFetchConfig(cfg), metrics, Fetch, time, threadNamePrefix, clientQuotaCallback),
|
||||
new ClientQuotaManager(clientProduceConfig(cfg), metrics, Produce, time, threadNamePrefix, clientQuotaCallback),
|
||||
new ClientRequestQuotaManager(clientRequestConfig(cfg), metrics, time, threadNamePrefix, clientQuotaCallback),
|
||||
new ControllerMutationQuotaManager(clientControllerMutationConfig(cfg), metrics, time,
|
||||
threadNamePrefix, clientQuotaCallback),
|
||||
new ReplicationQuotaManager(replicationConfig(cfg), metrics, LeaderReplication, time),
|
||||
new ReplicationQuotaManager(replicationConfig(cfg), metrics, FollowerReplication, time),
|
||||
new ReplicationQuotaManager(alterLogDirsReplicationConfig(cfg), metrics, AlterLogDirsReplication, time),
|
||||
|
@ -75,7 +92,7 @@ object QuotaFactory extends Logging {
|
|||
if (cfg.producerQuotaBytesPerSecondDefault != Long.MaxValue)
|
||||
warn(s"${KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp} has been deprecated in 0.11.0.0 and will be removed in a future release. Use dynamic quota defaults instead.")
|
||||
ClientQuotaManagerConfig(
|
||||
quotaBytesPerSecondDefault = cfg.producerQuotaBytesPerSecondDefault,
|
||||
quotaDefault = cfg.producerQuotaBytesPerSecondDefault,
|
||||
numQuotaSamples = cfg.numQuotaSamples,
|
||||
quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds
|
||||
)
|
||||
|
@ -85,7 +102,7 @@ object QuotaFactory extends Logging {
|
|||
if (cfg.consumerQuotaBytesPerSecondDefault != Long.MaxValue)
|
||||
warn(s"${KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp} has been deprecated in 0.11.0.0 and will be removed in a future release. Use dynamic quota defaults instead.")
|
||||
ClientQuotaManagerConfig(
|
||||
quotaBytesPerSecondDefault = cfg.consumerQuotaBytesPerSecondDefault,
|
||||
quotaDefault = cfg.consumerQuotaBytesPerSecondDefault,
|
||||
numQuotaSamples = cfg.numQuotaSamples,
|
||||
quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds
|
||||
)
|
||||
|
@ -98,6 +115,13 @@ object QuotaFactory extends Logging {
|
|||
)
|
||||
}
|
||||
|
||||
def clientControllerMutationConfig(cfg: KafkaConfig): ClientQuotaManagerConfig = {
|
||||
ClientQuotaManagerConfig(
|
||||
numQuotaSamples = cfg.numControllerQuotaSamples,
|
||||
quotaWindowSizeSeconds = cfg.controllerQuotaWindowSizeSeconds
|
||||
)
|
||||
}
|
||||
|
||||
def replicationConfig(cfg: KafkaConfig): ReplicationQuotaManagerConfig = {
|
||||
ReplicationQuotaManagerConfig(
|
||||
numQuotaSamples = cfg.numReplicationQuotaSamples,
|
||||
|
|
|
@ -81,10 +81,21 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
|
|||
brokerMetadatas.sortBy(_.id)
|
||||
}
|
||||
|
||||
/**
|
||||
* Create topic and optionally validate its parameters. Note that this method is used by the
|
||||
* TopicCommand as well.
|
||||
*
|
||||
* @param topic The name of the topic
|
||||
* @param config The config of the topic
|
||||
* @param partitionReplicaAssignment The assignments of the topic
|
||||
* @param validate Boolean indicating if parameters must be validated or not (true by default)
|
||||
*/
|
||||
def createTopicWithAssignment(topic: String,
|
||||
config: Properties,
|
||||
partitionReplicaAssignment: Map[Int, Seq[Int]]): Unit = {
|
||||
validateTopicCreate(topic, partitionReplicaAssignment, config)
|
||||
partitionReplicaAssignment: Map[Int, Seq[Int]],
|
||||
validate: Boolean = true): Unit = {
|
||||
if (validate)
|
||||
validateTopicCreate(topic, partitionReplicaAssignment, config)
|
||||
|
||||
info(s"Creating topic $topic with configuration $config and initial partition " +
|
||||
s"assignment $partitionReplicaAssignment")
|
||||
|
@ -98,7 +109,12 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Validate topic creation parameters
|
||||
* Validate topic creation parameters. Note that this method is indirectly used by the
|
||||
* TopicCommand via the `createTopicWithAssignment` method.
|
||||
*
|
||||
* @param topic The name of the topic
|
||||
* @param partitionReplicaAssignment The assignments of the topic
|
||||
* @param config The config of the topic
|
||||
*/
|
||||
def validateTopicCreate(topic: String,
|
||||
partitionReplicaAssignment: Map[Int, Seq[Int]],
|
||||
|
@ -171,22 +187,57 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Add partitions to existing topic with optional replica assignment
|
||||
*
|
||||
* @param topic Topic for adding partitions to
|
||||
* @param existingAssignment A map from partition id to its assignment
|
||||
* @param allBrokers All brokers in the cluster
|
||||
* @param numPartitions Number of partitions to be set
|
||||
* @param replicaAssignment Manual replica assignment, or none
|
||||
* @param validateOnly If true, validate the parameters without actually adding the partitions
|
||||
* @return the updated replica assignment
|
||||
*/
|
||||
* Add partitions to existing topic with optional replica assignment. Note that this
|
||||
* method is used by the TopicCommand.
|
||||
*
|
||||
* @param topic Topic for adding partitions to
|
||||
* @param existingAssignment A map from partition id to its assignment
|
||||
* @param allBrokers All brokers in the cluster
|
||||
* @param numPartitions Number of partitions to be set
|
||||
* @param replicaAssignment Manual replica assignment, or none
|
||||
* @param validateOnly If true, validate the parameters without actually adding the partitions
|
||||
* @return the updated replica assignment
|
||||
*/
|
||||
def addPartitions(topic: String,
|
||||
existingAssignment: Map[Int, ReplicaAssignment],
|
||||
allBrokers: Seq[BrokerMetadata],
|
||||
numPartitions: Int = 1,
|
||||
replicaAssignment: Option[Map[Int, Seq[Int]]] = None,
|
||||
validateOnly: Boolean = false): Map[Int, Seq[Int]] = {
|
||||
|
||||
val proposedAssignmentForNewPartitions = createNewPartitionsAssignment(
|
||||
topic,
|
||||
existingAssignment,
|
||||
allBrokers,
|
||||
numPartitions,
|
||||
replicaAssignment
|
||||
)
|
||||
|
||||
if (validateOnly) {
|
||||
(existingAssignment ++ proposedAssignmentForNewPartitions)
|
||||
.map { case (k, v) => k -> v.replicas }
|
||||
} else {
|
||||
createPartitionsWithAssignment(topic, existingAssignment, proposedAssignmentForNewPartitions)
|
||||
.map { case (k, v) => k -> v.replicas }
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create assignment to add the given number of partitions while validating the
|
||||
* provided arguments.
|
||||
*
|
||||
* @param topic Topic for adding partitions to
|
||||
* @param existingAssignment A map from partition id to its assignment
|
||||
* @param allBrokers All brokers in the cluster
|
||||
* @param numPartitions Number of partitions to be set
|
||||
* @param replicaAssignment Manual replica assignment, or none
|
||||
* @return the assignment for the new partitions
|
||||
*/
|
||||
def createNewPartitionsAssignment(topic: String,
|
||||
existingAssignment: Map[Int, ReplicaAssignment],
|
||||
allBrokers: Seq[BrokerMetadata],
|
||||
numPartitions: Int = 1,
|
||||
replicaAssignment: Option[Map[Int, Seq[Int]]] = None): Map[Int, ReplicaAssignment] = {
|
||||
val existingAssignmentPartition0 = existingAssignment.getOrElse(0,
|
||||
throw new AdminOperationException(
|
||||
s"Unexpected existing replica assignment for topic '$topic', partition id 0 is missing. " +
|
||||
|
@ -210,16 +261,32 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
|
|||
startIndex, existingAssignment.size)
|
||||
}
|
||||
|
||||
val proposedAssignment = existingAssignment ++ proposedAssignmentForNewPartitions.map { case (tp, replicas) =>
|
||||
proposedAssignmentForNewPartitions.map { case (tp, replicas) =>
|
||||
tp -> ReplicaAssignment(replicas, List(), List())
|
||||
}
|
||||
if (!validateOnly) {
|
||||
info(s"Creating $partitionsToAdd partitions for '$topic' with the following replica assignment: " +
|
||||
s"$proposedAssignmentForNewPartitions.")
|
||||
}
|
||||
|
||||
writeTopicPartitionAssignment(topic, proposedAssignment, isUpdate = true)
|
||||
}
|
||||
proposedAssignment.map { case (k, v) => k -> v.replicas }
|
||||
/**
|
||||
* Add partitions to the existing topic with the provided assignment. This method does
|
||||
* not validate the provided assignments. Validation must be done beforehand.
|
||||
*
|
||||
* @param topic Topic for adding partitions to
|
||||
* @param existingAssignment A map from partition id to its assignment
|
||||
* @param newPartitionAssignment The assignments to add
|
||||
* @return the updated replica assignment
|
||||
*/
|
||||
def createPartitionsWithAssignment(topic: String,
|
||||
existingAssignment: Map[Int, ReplicaAssignment],
|
||||
newPartitionAssignment: Map[Int, ReplicaAssignment]): Map[Int, ReplicaAssignment] = {
|
||||
|
||||
info(s"Creating ${newPartitionAssignment.size} partitions for '$topic' with the following replica assignment: " +
|
||||
s"$newPartitionAssignment.")
|
||||
|
||||
val combinedAssignment = existingAssignment ++ newPartitionAssignment
|
||||
|
||||
writeTopicPartitionAssignment(topic, combinedAssignment, isUpdate = true)
|
||||
|
||||
combinedAssignment
|
||||
}
|
||||
|
||||
private def validateReplicaAssignment(replicaAssignment: Map[Int, Seq[Int]],
|
||||
|
|
|
@ -1601,7 +1601,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
|||
"",
|
||||
"user", "",
|
||||
"client-id", clientId)
|
||||
assertNull("Metric should not hanve been created " + metricName, broker.metrics.metric(metricName))
|
||||
assertNull("Metric should not have been created " + metricName, broker.metrics.metric(metricName))
|
||||
}
|
||||
servers.foreach(assertNoMetric(_, "byte-rate", QuotaType.Produce, producerClientId))
|
||||
servers.foreach(assertNoMetric(_, "throttle-time", QuotaType.Produce, producerClientId))
|
||||
|
@ -1615,9 +1615,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
|||
|
||||
def assertNoExemptRequestMetric(broker: KafkaServer): Unit = {
|
||||
val metricName = broker.metrics.metricName("exempt-request-time", QuotaType.Request.toString, "")
|
||||
assertNull("Metric should not hanve been created " + metricName, broker.metrics.metric(metricName))
|
||||
assertNull("Metric should not have been created " + metricName, broker.metrics.metric(metricName))
|
||||
}
|
||||
servers.foreach(assertNoExemptRequestMetric(_))
|
||||
servers.foreach(assertNoExemptRequestMetric)
|
||||
}
|
||||
|
||||
def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = {
|
||||
|
|
|
@ -0,0 +1,92 @@
|
|||
/**
|
||||
* 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 java.net.InetAddress
|
||||
import java.util
|
||||
import java.util.Collections
|
||||
|
||||
import kafka.network.RequestChannel
|
||||
import kafka.network.RequestChannel.EndThrottlingResponse
|
||||
import kafka.network.RequestChannel.Session
|
||||
import kafka.network.RequestChannel.StartThrottlingResponse
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.memory.MemoryPool
|
||||
import org.apache.kafka.common.metrics.MetricConfig
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
import org.apache.kafka.common.network.ClientInformation
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.requests.AbstractRequest
|
||||
import org.apache.kafka.common.requests.FetchRequest
|
||||
import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
||||
import org.apache.kafka.common.requests.RequestContext
|
||||
import org.apache.kafka.common.requests.RequestHeader
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.MockTime
|
||||
import org.easymock.EasyMock
|
||||
import org.junit.After
|
||||
|
||||
class BaseClientQuotaManagerTest {
|
||||
protected val time = new MockTime
|
||||
protected var numCallbacks: Int = 0
|
||||
protected val metrics = new Metrics(new MetricConfig(), Collections.emptyList(), time)
|
||||
|
||||
@After
|
||||
def tearDown(): Unit = {
|
||||
metrics.close()
|
||||
}
|
||||
|
||||
protected def callback(response: RequestChannel.Response): Unit = {
|
||||
// Count how many times this callback is called for notifyThrottlingDone().
|
||||
response match {
|
||||
case _: StartThrottlingResponse =>
|
||||
case _: EndThrottlingResponse => numCallbacks += 1
|
||||
}
|
||||
}
|
||||
|
||||
protected def buildRequest[T <: AbstractRequest](builder: AbstractRequest.Builder[T],
|
||||
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)): (T, RequestChannel.Request) = {
|
||||
|
||||
val request = builder.build()
|
||||
val buffer = request.serialize(new RequestHeader(builder.apiKey, request.version, "", 0))
|
||||
val requestChannelMetrics: RequestChannel.Metrics = EasyMock.createNiceMock(classOf[RequestChannel.Metrics])
|
||||
|
||||
// read the header from the buffer first so that the body can be read next from the Request constructor
|
||||
val header = RequestHeader.parse(buffer)
|
||||
val context = new RequestContext(header, "1", InetAddress.getLocalHost, KafkaPrincipal.ANONYMOUS,
|
||||
listenerName, SecurityProtocol.PLAINTEXT, ClientInformation.EMPTY)
|
||||
(request, new RequestChannel.Request(processor = 1, context = context, startTimeNanos = 0, MemoryPool.NONE, buffer,
|
||||
requestChannelMetrics))
|
||||
}
|
||||
|
||||
protected def buildSession(user: String): Session = {
|
||||
val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user)
|
||||
Session(principal, null)
|
||||
}
|
||||
|
||||
protected def maybeRecord(quotaManager: ClientQuotaManager, user: String, clientId: String, value: Double): Int = {
|
||||
|
||||
quotaManager.maybeRecordAndGetThrottleTimeMs(buildSession(user), clientId, value, time.milliseconds)
|
||||
}
|
||||
|
||||
protected def throttle(quotaManager: ClientQuotaManager, user: String, clientId: String, throttleTimeMs: Int,
|
||||
channelThrottlingCallback: RequestChannel.Response => Unit): Unit = {
|
||||
val (_, request) = buildRequest(FetchRequest.Builder.forConsumer(0, 1000, new util.HashMap[TopicPartition, PartitionData]))
|
||||
quotaManager.throttle(request, throttleTimeMs, channelThrottlingCallback)
|
||||
}
|
||||
}
|
|
@ -17,113 +17,61 @@
|
|||
package kafka.server
|
||||
|
||||
import java.net.InetAddress
|
||||
import java.util
|
||||
import java.util.Collections
|
||||
|
||||
import kafka.network.RequestChannel
|
||||
import kafka.network.RequestChannel.{EndThrottlingResponse, Session, StartThrottlingResponse}
|
||||
import kafka.network.RequestChannel.Session
|
||||
import kafka.server.QuotaType._
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.memory.MemoryPool
|
||||
import org.apache.kafka.common.metrics.{MetricConfig, Metrics, Quota}
|
||||
import org.apache.kafka.common.network.ClientInformation
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
||||
import org.apache.kafka.common.requests.{AbstractRequest, FetchRequest, RequestContext, RequestHeader}
|
||||
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
|
||||
import org.apache.kafka.common.utils.{MockTime, Sanitizer}
|
||||
import org.easymock.EasyMock
|
||||
import org.junit.Assert.{assertEquals, assertTrue}
|
||||
import org.junit.{After, Test}
|
||||
import org.apache.kafka.common.metrics.Quota
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.common.utils.Sanitizer
|
||||
|
||||
class ClientQuotaManagerTest {
|
||||
private val time = new MockTime
|
||||
private val metrics = new Metrics(new MetricConfig(), Collections.emptyList(), time)
|
||||
private val config = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500)
|
||||
import org.junit.Assert._
|
||||
import org.junit.Test
|
||||
|
||||
var numCallbacks: Int = 0
|
||||
|
||||
@After
|
||||
def tearDown(): Unit = {
|
||||
metrics.close()
|
||||
}
|
||||
|
||||
def callback (response: RequestChannel.Response): Unit = {
|
||||
// Count how many times this callback is called for notifyThrottlingDone().
|
||||
response match {
|
||||
case _: StartThrottlingResponse =>
|
||||
case _: EndThrottlingResponse => numCallbacks += 1
|
||||
}
|
||||
}
|
||||
|
||||
private def buildRequest[T <: AbstractRequest](builder: AbstractRequest.Builder[T],
|
||||
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)): (T, RequestChannel.Request) = {
|
||||
|
||||
val request = builder.build()
|
||||
val buffer = request.serialize(new RequestHeader(builder.apiKey, request.version, "", 0))
|
||||
val requestChannelMetrics: RequestChannel.Metrics = EasyMock.createNiceMock(classOf[RequestChannel.Metrics])
|
||||
|
||||
// read the header from the buffer first so that the body can be read next from the Request constructor
|
||||
val header = RequestHeader.parse(buffer)
|
||||
val context = new RequestContext(header, "1", InetAddress.getLocalHost, KafkaPrincipal.ANONYMOUS,
|
||||
listenerName, SecurityProtocol.PLAINTEXT, ClientInformation.EMPTY)
|
||||
(request, new RequestChannel.Request(processor = 1, context = context, startTimeNanos = 0, MemoryPool.NONE, buffer,
|
||||
requestChannelMetrics))
|
||||
}
|
||||
|
||||
private def maybeRecord(quotaManager: ClientQuotaManager, user: String, clientId: String, value: Double): Int = {
|
||||
val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user)
|
||||
quotaManager.maybeRecordAndGetThrottleTimeMs(Session(principal, null), clientId, value, time.milliseconds())
|
||||
}
|
||||
|
||||
private def throttle(quotaManager: ClientQuotaManager, user: String, clientId: String, throttleTimeMs: Int,
|
||||
channelThrottlingCallback: (RequestChannel.Response) => Unit): Unit = {
|
||||
val (_, request) = buildRequest(FetchRequest.Builder.forConsumer(0, 1000, new util.HashMap[TopicPartition, PartitionData]))
|
||||
quotaManager.throttle(request, throttleTimeMs, channelThrottlingCallback)
|
||||
}
|
||||
class ClientQuotaManagerTest extends BaseClientQuotaManagerTest {
|
||||
private val config = ClientQuotaManagerConfig(quotaDefault = 500)
|
||||
|
||||
private def testQuotaParsing(config: ClientQuotaManagerConfig, client1: UserClient, client2: UserClient, randomClient: UserClient, defaultConfigClient: UserClient): Unit = {
|
||||
val clientMetrics = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
|
||||
try {
|
||||
// Case 1: Update the quota. Assert that the new quota value is returned
|
||||
clientMetrics.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(2000, true)))
|
||||
clientMetrics.updateQuota(client2.configUser, client2.configClientId, client2.sanitizedConfigClientId, Some(new Quota(4000, true)))
|
||||
clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(2000, true)))
|
||||
clientQuotaManager.updateQuota(client2.configUser, client2.configClientId, client2.sanitizedConfigClientId, Some(new Quota(4000, true)))
|
||||
|
||||
assertEquals("Default producer quota should be " + config.quotaBytesPerSecondDefault,
|
||||
config.quotaBytesPerSecondDefault.toDouble, clientMetrics.quota(randomClient.user, randomClient.clientId).bound, 0.0)
|
||||
assertEquals("Should return the overridden value (2000)", 2000, clientMetrics.quota(client1.user, client1.clientId).bound, 0.0)
|
||||
assertEquals("Should return the overridden value (4000)", 4000, clientMetrics.quota(client2.user, client2.clientId).bound, 0.0)
|
||||
assertEquals("Default producer quota should be " + config.quotaDefault,
|
||||
config.quotaDefault.toDouble, clientQuotaManager.quota(randomClient.user, randomClient.clientId).bound, 0.0)
|
||||
assertEquals("Should return the overridden value (2000)", 2000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0)
|
||||
assertEquals("Should return the overridden value (4000)", 4000, clientQuotaManager.quota(client2.user, client2.clientId).bound, 0.0)
|
||||
|
||||
// p1 should be throttled using the overridden quota
|
||||
var throttleTimeMs = maybeRecord(clientMetrics, client1.user, client1.clientId, 2500 * config.numQuotaSamples)
|
||||
var throttleTimeMs = maybeRecord(clientQuotaManager, client1.user, client1.clientId, 2500 * config.numQuotaSamples)
|
||||
assertTrue(s"throttleTimeMs should be > 0. was $throttleTimeMs", throttleTimeMs > 0)
|
||||
|
||||
// Case 2: Change quota again. The quota should be updated within KafkaMetrics as well since the sensor was created.
|
||||
// p1 should not longer be throttled after the quota change
|
||||
clientMetrics.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(3000, true)))
|
||||
assertEquals("Should return the newly overridden value (3000)", 3000, clientMetrics.quota(client1.user, client1.clientId).bound, 0.0)
|
||||
clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(3000, true)))
|
||||
assertEquals("Should return the newly overridden value (3000)", 3000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0)
|
||||
|
||||
throttleTimeMs = maybeRecord(clientMetrics, client1.user, client1.clientId, 0)
|
||||
throttleTimeMs = maybeRecord(clientQuotaManager, client1.user, client1.clientId, 0)
|
||||
assertEquals(s"throttleTimeMs should be 0. was $throttleTimeMs", 0, throttleTimeMs)
|
||||
|
||||
// Case 3: Change quota back to default. Should be throttled again
|
||||
clientMetrics.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(500, true)))
|
||||
assertEquals("Should return the default value (500)", 500, clientMetrics.quota(client1.user, client1.clientId).bound, 0.0)
|
||||
clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(500, true)))
|
||||
assertEquals("Should return the default value (500)", 500, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0)
|
||||
|
||||
throttleTimeMs = maybeRecord(clientMetrics, client1.user, client1.clientId, 0)
|
||||
throttleTimeMs = maybeRecord(clientQuotaManager, client1.user, client1.clientId, 0)
|
||||
assertTrue(s"throttleTimeMs should be > 0. was $throttleTimeMs", throttleTimeMs > 0)
|
||||
|
||||
// Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled
|
||||
clientMetrics.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, None)
|
||||
clientMetrics.updateQuota(defaultConfigClient.configUser, defaultConfigClient.configClientId, defaultConfigClient.sanitizedConfigClientId, Some(new Quota(4000, true)))
|
||||
assertEquals("Should return the newly overridden value (4000)", 4000, clientMetrics.quota(client1.user, client1.clientId).bound, 0.0)
|
||||
clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, None)
|
||||
clientQuotaManager.updateQuota(defaultConfigClient.configUser, defaultConfigClient.configClientId, defaultConfigClient.sanitizedConfigClientId, Some(new Quota(4000, true)))
|
||||
assertEquals("Should return the newly overridden value (4000)", 4000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0)
|
||||
|
||||
throttleTimeMs = maybeRecord(clientMetrics, client1.user, client1.clientId, 1000 * config.numQuotaSamples)
|
||||
throttleTimeMs = maybeRecord(clientQuotaManager, client1.user, client1.clientId, 1000 * config.numQuotaSamples)
|
||||
assertEquals(s"throttleTimeMs should be 0. was $throttleTimeMs", 0, throttleTimeMs)
|
||||
|
||||
} finally {
|
||||
clientMetrics.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -150,7 +98,7 @@ class ClientQuotaManagerTest {
|
|||
val client2 = UserClient("User2", "p2", Some("User2"), None)
|
||||
val randomClient = UserClient("RandomUser", "random-client-id", None, None)
|
||||
val defaultConfigClient = UserClient("", "", Some(ConfigEntityName.Default), None)
|
||||
val config = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = Long.MaxValue)
|
||||
val config = ClientQuotaManagerConfig(quotaDefault = Long.MaxValue)
|
||||
testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient)
|
||||
}
|
||||
|
||||
|
@ -164,7 +112,7 @@ class ClientQuotaManagerTest {
|
|||
val client2 = UserClient("User2", "p2", Some("User2"), Some("p2"))
|
||||
val randomClient = UserClient("RandomUser", "random-client-id", None, None)
|
||||
val defaultConfigClient = UserClient("", "", Some(ConfigEntityName.Default), Some(ConfigEntityName.Default))
|
||||
val config = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = Long.MaxValue)
|
||||
val config = ClientQuotaManagerConfig(quotaDefault = Long.MaxValue)
|
||||
testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient)
|
||||
}
|
||||
|
||||
|
@ -210,156 +158,155 @@ class ClientQuotaManagerTest {
|
|||
@Test
|
||||
def testGetMaxValueInQuotaWindowWithNonDefaultQuotaWindow(): Unit = {
|
||||
val numFullQuotaWindows = 3 // 3 seconds window (vs. 10 seconds default)
|
||||
val nonDefaultConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = Long.MaxValue, numQuotaSamples = numFullQuotaWindows + 1)
|
||||
val quotaManager = new ClientQuotaManager(nonDefaultConfig, metrics, Fetch, time, "")
|
||||
val nonDefaultConfig = ClientQuotaManagerConfig(quotaDefault = Long.MaxValue, numQuotaSamples = numFullQuotaWindows + 1)
|
||||
val clientQuotaManager = new ClientQuotaManager(nonDefaultConfig, metrics, Fetch, time, "")
|
||||
val userSession = Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "userA"), InetAddress.getLocalHost)
|
||||
|
||||
try {
|
||||
// no quota set
|
||||
assertEquals(Double.MaxValue, quotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01)
|
||||
assertEquals(Double.MaxValue, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01)
|
||||
|
||||
// Set default <user> quota config
|
||||
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(10, true)))
|
||||
assertEquals(10 * numFullQuotaWindows, quotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01)
|
||||
clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(10, true)))
|
||||
assertEquals(10 * numFullQuotaWindows, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01)
|
||||
} finally {
|
||||
quotaManager.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testSetAndRemoveDefaultUserQuota(): Unit = {
|
||||
// quotaTypesEnabled will be QuotaTypes.NoQuotas initially
|
||||
val quotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaBytesPerSecondDefault = Long.MaxValue),
|
||||
val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault = Long.MaxValue),
|
||||
metrics, Produce, time, "")
|
||||
|
||||
try {
|
||||
// no quota set yet, should not throttle
|
||||
checkQuota(quotaManager, "userA", "client1", Long.MaxValue, 1000, false)
|
||||
checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, false)
|
||||
|
||||
// Set default <user> quota config
|
||||
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(10, true)))
|
||||
checkQuota(quotaManager, "userA", "client1", 10, 1000, true)
|
||||
clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(10, true)))
|
||||
checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, true)
|
||||
|
||||
// Remove default <user> quota config, back to no quotas
|
||||
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, None)
|
||||
checkQuota(quotaManager, "userA", "client1", Long.MaxValue, 1000, false)
|
||||
clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, None)
|
||||
checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, false)
|
||||
} finally {
|
||||
quotaManager.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testSetAndRemoveUserQuota(): Unit = {
|
||||
// quotaTypesEnabled will be QuotaTypes.NoQuotas initially
|
||||
val quotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaBytesPerSecondDefault = Long.MaxValue),
|
||||
val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault = Long.MaxValue),
|
||||
metrics, Produce, time, "")
|
||||
|
||||
try {
|
||||
// Set <user> quota config
|
||||
quotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(10, true)))
|
||||
checkQuota(quotaManager, "userA", "client1", 10, 1000, true)
|
||||
clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(10, true)))
|
||||
checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, true)
|
||||
|
||||
// Remove <user> quota config, back to no quotas
|
||||
quotaManager.updateQuota(Some("userA"), None, None, None)
|
||||
checkQuota(quotaManager, "userA", "client1", Long.MaxValue, 1000, false)
|
||||
clientQuotaManager.updateQuota(Some("userA"), None, None, None)
|
||||
checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, false)
|
||||
} finally {
|
||||
quotaManager.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testSetAndRemoveUserClientQuota(): Unit = {
|
||||
// quotaTypesEnabled will be QuotaTypes.NoQuotas initially
|
||||
val quotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaBytesPerSecondDefault = Long.MaxValue),
|
||||
val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault = Long.MaxValue),
|
||||
metrics, Produce, time, "")
|
||||
|
||||
try {
|
||||
// Set <user, client-id> quota config
|
||||
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10, true)))
|
||||
checkQuota(quotaManager, "userA", "client1", 10, 1000, true)
|
||||
clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10, true)))
|
||||
checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, true)
|
||||
|
||||
// Remove <user, client-id> quota config, back to no quotas
|
||||
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None)
|
||||
checkQuota(quotaManager, "userA", "client1", Long.MaxValue, 1000, false)
|
||||
clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None)
|
||||
checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, false)
|
||||
} finally {
|
||||
quotaManager.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testQuotaConfigPrecedence(): Unit = {
|
||||
val quotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaBytesPerSecondDefault=Long.MaxValue),
|
||||
val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault=Long.MaxValue),
|
||||
metrics, Produce, time, "")
|
||||
|
||||
try {
|
||||
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(1000, true)))
|
||||
quotaManager.updateQuota(None, Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(2000, true)))
|
||||
quotaManager.updateQuota(Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(3000, true)))
|
||||
quotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(4000, true)))
|
||||
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(5000, true)))
|
||||
quotaManager.updateQuota(Some("userB"), None, None, Some(new Quota(6000, true)))
|
||||
quotaManager.updateQuota(Some("userB"), Some("client1"), Some("client1"), Some(new Quota(7000, true)))
|
||||
quotaManager.updateQuota(Some("userB"), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(8000, true)))
|
||||
quotaManager.updateQuota(Some("userC"), None, None, Some(new Quota(10000, true)))
|
||||
quotaManager.updateQuota(None, Some("client1"), Some("client1"), Some(new Quota(9000, true)))
|
||||
clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(1000, true)))
|
||||
clientQuotaManager.updateQuota(None, Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(2000, true)))
|
||||
clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(3000, true)))
|
||||
clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(4000, true)))
|
||||
clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(5000, true)))
|
||||
clientQuotaManager.updateQuota(Some("userB"), None, None, Some(new Quota(6000, true)))
|
||||
clientQuotaManager.updateQuota(Some("userB"), Some("client1"), Some("client1"), Some(new Quota(7000, true)))
|
||||
clientQuotaManager.updateQuota(Some("userB"), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(8000, true)))
|
||||
clientQuotaManager.updateQuota(Some("userC"), None, None, Some(new Quota(10000, true)))
|
||||
clientQuotaManager.updateQuota(None, Some("client1"), Some("client1"), Some(new Quota(9000, true)))
|
||||
|
||||
checkQuota(quotaManager, "userA", "client1", 5000, 4500, false) // <user, client> quota takes precedence over <user>
|
||||
checkQuota(quotaManager, "userA", "client2", 4000, 4500, true) // <user> quota takes precedence over <client> and defaults
|
||||
checkQuota(quotaManager, "userA", "client3", 4000, 0, true) // <user> quota is shared across clients of user
|
||||
checkQuota(quotaManager, "userA", "client1", 5000, 0, false) // <user, client> is exclusive use, unaffected by other clients
|
||||
checkQuota(clientQuotaManager, "userA", "client1", 5000, 4500, false) // <user, client> quota takes precedence over <user>
|
||||
checkQuota(clientQuotaManager, "userA", "client2", 4000, 4500, true) // <user> quota takes precedence over <client> and defaults
|
||||
checkQuota(clientQuotaManager, "userA", "client3", 4000, 0, true) // <user> quota is shared across clients of user
|
||||
checkQuota(clientQuotaManager, "userA", "client1", 5000, 0, false) // <user, client> is exclusive use, unaffected by other clients
|
||||
|
||||
checkQuota(quotaManager, "userB", "client1", 7000, 8000, true)
|
||||
checkQuota(quotaManager, "userB", "client2", 8000, 7000, false) // Default per-client quota for exclusive use of <user, client>
|
||||
checkQuota(quotaManager, "userB", "client3", 8000, 7000, false)
|
||||
checkQuota(clientQuotaManager, "userB", "client1", 7000, 8000, true)
|
||||
checkQuota(clientQuotaManager, "userB", "client2", 8000, 7000, false) // Default per-client quota for exclusive use of <user, client>
|
||||
checkQuota(clientQuotaManager, "userB", "client3", 8000, 7000, false)
|
||||
|
||||
checkQuota(quotaManager, "userD", "client1", 3000, 3500, true) // Default <user, client> quota
|
||||
checkQuota(quotaManager, "userD", "client2", 3000, 2500, false)
|
||||
checkQuota(quotaManager, "userE", "client1", 3000, 2500, false)
|
||||
checkQuota(clientQuotaManager, "userD", "client1", 3000, 3500, true) // Default <user, client> quota
|
||||
checkQuota(clientQuotaManager, "userD", "client2", 3000, 2500, false)
|
||||
checkQuota(clientQuotaManager, "userE", "client1", 3000, 2500, false)
|
||||
|
||||
// Remove default <user, client> quota config, revert to <user> default
|
||||
quotaManager.updateQuota(Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), None)
|
||||
checkQuota(quotaManager, "userD", "client1", 1000, 0, false) // Metrics tags changed, restart counter
|
||||
checkQuota(quotaManager, "userE", "client4", 1000, 1500, true)
|
||||
checkQuota(quotaManager, "userF", "client4", 1000, 800, false) // Default <user> quota shared across clients of user
|
||||
checkQuota(quotaManager, "userF", "client5", 1000, 800, true)
|
||||
clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), None)
|
||||
checkQuota(clientQuotaManager, "userD", "client1", 1000, 0, false) // Metrics tags changed, restart counter
|
||||
checkQuota(clientQuotaManager, "userE", "client4", 1000, 1500, true)
|
||||
checkQuota(clientQuotaManager, "userF", "client4", 1000, 800, false) // Default <user> quota shared across clients of user
|
||||
checkQuota(clientQuotaManager, "userF", "client5", 1000, 800, true)
|
||||
|
||||
// Remove default <user> quota config, revert to <client-id> default
|
||||
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, None)
|
||||
checkQuota(quotaManager, "userF", "client4", 2000, 0, false) // Default <client-id> quota shared across client-id of all users
|
||||
checkQuota(quotaManager, "userF", "client5", 2000, 0, false)
|
||||
checkQuota(quotaManager, "userF", "client5", 2000, 2500, true)
|
||||
checkQuota(quotaManager, "userG", "client5", 2000, 0, true)
|
||||
clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, None)
|
||||
checkQuota(clientQuotaManager, "userF", "client4", 2000, 0, false) // Default <client-id> quota shared across client-id of all users
|
||||
checkQuota(clientQuotaManager, "userF", "client5", 2000, 0, false)
|
||||
checkQuota(clientQuotaManager, "userF", "client5", 2000, 2500, true)
|
||||
checkQuota(clientQuotaManager, "userG", "client5", 2000, 0, true)
|
||||
|
||||
// Update quotas
|
||||
quotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(8000, true)))
|
||||
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10000, true)))
|
||||
checkQuota(quotaManager, "userA", "client2", 8000, 0, false)
|
||||
checkQuota(quotaManager, "userA", "client2", 8000, 4500, true) // Throttled due to sum of new and earlier values
|
||||
checkQuota(quotaManager, "userA", "client1", 10000, 0, false)
|
||||
checkQuota(quotaManager, "userA", "client1", 10000, 6000, true)
|
||||
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None)
|
||||
checkQuota(quotaManager, "userA", "client6", 8000, 0, true) // Throttled due to shared user quota
|
||||
quotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), Some(new Quota(11000, true)))
|
||||
checkQuota(quotaManager, "userA", "client6", 11000, 8500, false)
|
||||
quotaManager.updateQuota(Some("userA"), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(12000, true)))
|
||||
quotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), None)
|
||||
checkQuota(quotaManager, "userA", "client6", 12000, 4000, true) // Throttled due to sum of new and earlier values
|
||||
clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(8000, true)))
|
||||
clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10000, true)))
|
||||
checkQuota(clientQuotaManager, "userA", "client2", 8000, 0, false)
|
||||
checkQuota(clientQuotaManager, "userA", "client2", 8000, 4500, true) // Throttled due to sum of new and earlier values
|
||||
checkQuota(clientQuotaManager, "userA", "client1", 10000, 0, false)
|
||||
checkQuota(clientQuotaManager, "userA", "client1", 10000, 6000, true)
|
||||
clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None)
|
||||
checkQuota(clientQuotaManager, "userA", "client6", 8000, 0, true) // Throttled due to shared user quota
|
||||
clientQuotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), Some(new Quota(11000, true)))
|
||||
checkQuota(clientQuotaManager, "userA", "client6", 11000, 8500, false)
|
||||
clientQuotaManager.updateQuota(Some("userA"), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(12000, true)))
|
||||
clientQuotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), None)
|
||||
checkQuota(clientQuotaManager, "userA", "client6", 12000, 4000, true) // Throttled due to sum of new and earlier values
|
||||
|
||||
} finally {
|
||||
quotaManager.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testQuotaViolation(): Unit = {
|
||||
val clientMetrics = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", "Produce", ""))
|
||||
try {
|
||||
/* We have 10 second windows. Make sure that there is no quota violation
|
||||
* if we produce under the quota
|
||||
*/
|
||||
// We have 10 second windows. Make sure that there is no quota violation
|
||||
// if we produce under the quota
|
||||
for (_ <- 0 until 10) {
|
||||
assertEquals(0, maybeRecord(clientMetrics, "ANONYMOUS", "unknown", 400))
|
||||
assertEquals(0, maybeRecord(clientQuotaManager, "ANONYMOUS", "unknown", 400))
|
||||
time.sleep(1000)
|
||||
}
|
||||
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
|
@ -369,125 +316,63 @@ class ClientQuotaManagerTest {
|
|||
// (600 - quota)/quota*window-size = (600-500)/500*10.5 seconds = 2100
|
||||
// 10.5 seconds because the last window is half complete
|
||||
time.sleep(500)
|
||||
val sleepTime = maybeRecord(clientMetrics, "ANONYMOUS", "unknown", 2300)
|
||||
val throttleTime = maybeRecord(clientQuotaManager, "ANONYMOUS", "unknown", 2300)
|
||||
|
||||
assertEquals("Should be throttled", 2100, sleepTime)
|
||||
throttle(clientMetrics, "ANONYMOYUS", "unknown", sleepTime, callback)
|
||||
assertEquals("Should be throttled", 2100, throttleTime)
|
||||
throttle(clientQuotaManager, "ANONYMOUS", "unknown", throttleTime, callback)
|
||||
assertEquals(1, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
// After a request is delayed, the callback cannot be triggered immediately
|
||||
clientMetrics.throttledChannelReaper.doWork()
|
||||
assertEquals(0, numCallbacks)
|
||||
time.sleep(sleepTime)
|
||||
|
||||
// Callback can only be triggered after the delay time passes
|
||||
clientMetrics.throttledChannelReaper.doWork()
|
||||
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
assertEquals(1, numCallbacks)
|
||||
|
||||
// Could continue to see delays until the bursty sample disappears
|
||||
for (_ <- 0 until 10) {
|
||||
maybeRecord(clientMetrics, "ANONYMOUS", "unknown", 400)
|
||||
time.sleep(1000)
|
||||
}
|
||||
|
||||
assertEquals("Should be unthrottled since bursty sample has rolled over",
|
||||
0, maybeRecord(clientMetrics, "ANONYMOUS", "unknown", 0))
|
||||
} finally {
|
||||
clientMetrics.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testRequestPercentageQuotaViolation(): Unit = {
|
||||
val quotaManager = new ClientRequestQuotaManager(config, metrics, time, "", None)
|
||||
quotaManager.updateQuota(Some("ANONYMOUS"), Some("test-client"), Some("test-client"), Some(Quota.upperBound(1)))
|
||||
val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", "Request", ""))
|
||||
def millisToPercent(millis: Double) = millis * 1000 * 1000 * ClientQuotaManagerConfig.NanosToPercentagePerSecond
|
||||
try {
|
||||
/* We have 10 second windows. Make sure that there is no quota violation
|
||||
* if we are under the quota
|
||||
*/
|
||||
for (_ <- 0 until 10) {
|
||||
assertEquals(0, maybeRecord(quotaManager, "ANONYMOUS", "test-client", millisToPercent(4)))
|
||||
time.sleep(1000)
|
||||
}
|
||||
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
|
||||
// Create a spike.
|
||||
// quota = 1% (10ms per second)
|
||||
// 4*10 + 67.1 = 107.1/10.5 = 10.2ms per second.
|
||||
// (10.2 - quota)/quota*window-size = (10.2-10)/10*10.5 seconds = 210ms
|
||||
// 10.5 seconds interval because the last window is half complete
|
||||
time.sleep(500)
|
||||
val throttleTime = maybeRecord(quotaManager, "ANONYMOUS", "test-client", millisToPercent(67.1))
|
||||
|
||||
assertEquals("Should be throttled", 210, throttleTime)
|
||||
|
||||
throttle(quotaManager, "ANONYMOYUS", "test-client", throttleTime, callback)
|
||||
assertEquals(1, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
// After a request is delayed, the callback cannot be triggered immediately
|
||||
quotaManager.throttledChannelReaper.doWork()
|
||||
clientQuotaManager.throttledChannelReaper.doWork()
|
||||
assertEquals(0, numCallbacks)
|
||||
time.sleep(throttleTime)
|
||||
|
||||
// Callback can only be triggered after the delay time passes
|
||||
quotaManager.throttledChannelReaper.doWork()
|
||||
clientQuotaManager.throttledChannelReaper.doWork()
|
||||
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
assertEquals(1, numCallbacks)
|
||||
|
||||
// Could continue to see delays until the bursty sample disappears
|
||||
for (_ <- 0 until 11) {
|
||||
maybeRecord(quotaManager, "ANONYMOUS", "test-client", millisToPercent(4))
|
||||
time.sleep(1000)
|
||||
}
|
||||
|
||||
assertEquals("Should be unthrottled since bursty sample has rolled over",
|
||||
0, maybeRecord(quotaManager, "ANONYMOUS", "test-client", 0))
|
||||
|
||||
// Create a very large spike which requires > one quota window to bring within quota
|
||||
assertEquals(1000, maybeRecord(quotaManager, "ANONYMOUS", "test-client", millisToPercent(500)))
|
||||
for (_ <- 0 until 10) {
|
||||
maybeRecord(clientQuotaManager, "ANONYMOUS", "unknown", 400)
|
||||
time.sleep(1000)
|
||||
assertEquals(1000, maybeRecord(quotaManager, "ANONYMOUS", "test-client", 0))
|
||||
}
|
||||
time.sleep(1000)
|
||||
assertEquals("Should be unthrottled since bursty sample has rolled over",
|
||||
0, maybeRecord(quotaManager, "ANONYMOUS", "test-client", 0))
|
||||
|
||||
assertEquals("Should be unthrottled since bursty sample has rolled over",
|
||||
0, maybeRecord(clientQuotaManager, "ANONYMOUS", "unknown", 0))
|
||||
} finally {
|
||||
quotaManager.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testExpireThrottleTimeSensor(): Unit = {
|
||||
val clientMetrics = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
try {
|
||||
maybeRecord(clientMetrics, "ANONYMOUS", "client1", 100)
|
||||
maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100)
|
||||
// remove the throttle time sensor
|
||||
metrics.removeSensor("ProduceThrottleTime-:client1")
|
||||
// should not throw an exception even if the throttle time sensor does not exist.
|
||||
val throttleTime = maybeRecord(clientMetrics, "ANONYMOUS", "client1", 10000)
|
||||
val throttleTime = maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 10000)
|
||||
assertTrue("Should be throttled", throttleTime > 0)
|
||||
// the sensor should get recreated
|
||||
val throttleTimeSensor = metrics.getSensor("ProduceThrottleTime-:client1")
|
||||
assertTrue("Throttle time sensor should exist", throttleTimeSensor != null)
|
||||
assertTrue("Throttle time sensor should exist", throttleTimeSensor != null)
|
||||
} finally {
|
||||
clientMetrics.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testExpireQuotaSensors(): Unit = {
|
||||
val clientMetrics = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
try {
|
||||
maybeRecord(clientMetrics, "ANONYMOUS", "client1", 100)
|
||||
maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100)
|
||||
// remove all the sensors
|
||||
metrics.removeSensor("ProduceThrottleTime-:client1")
|
||||
metrics.removeSensor("Produce-ANONYMOUS:client1")
|
||||
// should not throw an exception
|
||||
val throttleTime = maybeRecord(clientMetrics, "ANONYMOUS", "client1", 10000)
|
||||
val throttleTime = maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 10000)
|
||||
assertTrue("Should be throttled", throttleTime > 0)
|
||||
|
||||
// all the sensors should get recreated
|
||||
|
@ -497,16 +382,16 @@ class ClientQuotaManagerTest {
|
|||
val byteRateSensor = metrics.getSensor("Produce-:client1")
|
||||
assertTrue("Byte rate sensor should exist", byteRateSensor != null)
|
||||
} finally {
|
||||
clientMetrics.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testClientIdNotSanitized(): Unit = {
|
||||
val clientMetrics = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "")
|
||||
val clientId = "client@#$%"
|
||||
try {
|
||||
maybeRecord(clientMetrics, "ANONYMOUS", clientId, 100)
|
||||
maybeRecord(clientQuotaManager, "ANONYMOUS", clientId, 100)
|
||||
|
||||
// The metrics should use the raw client ID, even if the reporters internally sanitize them
|
||||
val throttleTimeSensor = metrics.getSensor("ProduceThrottleTime-:" + clientId)
|
||||
|
@ -515,7 +400,7 @@ class ClientQuotaManagerTest {
|
|||
val byteRateSensor = metrics.getSensor("Produce-:" + clientId)
|
||||
assertTrue("Byte rate sensor should exist", byteRateSensor != null)
|
||||
} finally {
|
||||
clientMetrics.shutdown()
|
||||
clientQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,89 @@
|
|||
/**
|
||||
* 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.server.QuotaType.Request
|
||||
import org.apache.kafka.common.metrics.Quota
|
||||
|
||||
import org.junit.Assert._
|
||||
import org.junit.Test
|
||||
|
||||
class ClientRequestQuotaManagerTest extends BaseClientQuotaManagerTest {
|
||||
private val config = ClientQuotaManagerConfig()
|
||||
|
||||
@Test
|
||||
def testRequestPercentageQuotaViolation(): Unit = {
|
||||
val clientRequestQuotaManager = new ClientRequestQuotaManager(config, metrics, time, "", None)
|
||||
clientRequestQuotaManager.updateQuota(Some("ANONYMOUS"), Some("test-client"), Some("test-client"), Some(Quota.upperBound(1)))
|
||||
val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", Request.toString, ""))
|
||||
def millisToPercent(millis: Double) = millis * 1000 * 1000 * ClientRequestQuotaManager.NanosToPercentagePerSecond
|
||||
try {
|
||||
// We have 10 second windows. Make sure that there is no quota violation
|
||||
// if we are under the quota
|
||||
for (_ <- 0 until 10) {
|
||||
assertEquals(0, maybeRecord(clientRequestQuotaManager, "ANONYMOUS", "test-client", millisToPercent(4)))
|
||||
time.sleep(1000)
|
||||
}
|
||||
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
|
||||
// Create a spike.
|
||||
// quota = 1% (10ms per second)
|
||||
// 4*10 + 67.1 = 107.1/10.5 = 10.2ms per second.
|
||||
// (10.2 - quota)/quota*window-size = (10.2-10)/10*10.5 seconds = 210ms
|
||||
// 10.5 seconds interval because the last window is half complete
|
||||
time.sleep(500)
|
||||
val throttleTime = maybeRecord(clientRequestQuotaManager, "ANONYMOUS", "test-client", millisToPercent(67.1))
|
||||
|
||||
assertEquals("Should be throttled", 210, throttleTime)
|
||||
|
||||
throttle(clientRequestQuotaManager, "ANONYMOUS", "test-client", throttleTime, callback)
|
||||
assertEquals(1, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
// After a request is delayed, the callback cannot be triggered immediately
|
||||
clientRequestQuotaManager.throttledChannelReaper.doWork()
|
||||
assertEquals(0, numCallbacks)
|
||||
time.sleep(throttleTime)
|
||||
|
||||
// Callback can only be triggered after the delay time passes
|
||||
clientRequestQuotaManager.throttledChannelReaper.doWork()
|
||||
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
assertEquals(1, numCallbacks)
|
||||
|
||||
// Could continue to see delays until the bursty sample disappears
|
||||
for (_ <- 0 until 11) {
|
||||
maybeRecord(clientRequestQuotaManager, "ANONYMOUS", "test-client", millisToPercent(4))
|
||||
time.sleep(1000)
|
||||
}
|
||||
|
||||
assertEquals("Should be unthrottled since bursty sample has rolled over",
|
||||
0, maybeRecord(clientRequestQuotaManager, "ANONYMOUS", "test-client", 0))
|
||||
|
||||
// Create a very large spike which requires > one quota window to bring within quota
|
||||
assertEquals(1000, maybeRecord(clientRequestQuotaManager, "ANONYMOUS", "test-client", millisToPercent(500)))
|
||||
for (_ <- 0 until 10) {
|
||||
time.sleep(1000)
|
||||
assertEquals(1000, maybeRecord(clientRequestQuotaManager, "ANONYMOUS", "test-client", 0))
|
||||
}
|
||||
time.sleep(1000)
|
||||
assertEquals("Should be unthrottled since bursty sample has rolled over",
|
||||
0, maybeRecord(clientRequestQuotaManager, "ANONYMOUS", "test-client", 0))
|
||||
|
||||
} finally {
|
||||
clientRequestQuotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,216 @@
|
|||
/**
|
||||
* 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 java.util.concurrent.TimeUnit
|
||||
|
||||
import kafka.server.QuotaType.ControllerMutation
|
||||
import org.apache.kafka.common.errors.ThrottlingQuotaExceededException
|
||||
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.stats.Rate
|
||||
import org.apache.kafka.common.utils.MockTime
|
||||
import org.junit.Assert._
|
||||
import org.junit.Assert.assertEquals
|
||||
import org.junit.Assert.assertFalse
|
||||
import org.junit.Test
|
||||
|
||||
class StrictControllerMutationQuotaTest {
|
||||
@Test
|
||||
def testControllerMutationQuotaViolation(): Unit = {
|
||||
val time = new MockTime(0, System.currentTimeMillis, 0)
|
||||
val metrics = new Metrics(time)
|
||||
val sensor = metrics.sensor("sensor", new MetricConfig()
|
||||
.quota(Quota.upperBound(10))
|
||||
.timeWindow(1, TimeUnit.SECONDS)
|
||||
.samples(11))
|
||||
val metricName = metrics.metricName("rate", "test-group")
|
||||
assertTrue(sensor.add(metricName, new Rate))
|
||||
|
||||
val quota = new StrictControllerMutationQuota(time, sensor)
|
||||
assertFalse(quota.isExceeded)
|
||||
|
||||
// Recording a first value at T to bring the avg rate to 9. Value is accepted
|
||||
// because the quota is not exhausted yet.
|
||||
quota.record(90)
|
||||
assertFalse(quota.isExceeded)
|
||||
assertEquals(0, quota.throttleTime)
|
||||
|
||||
// Recording a second value at T to bring the avg rate to 18. Value is accepted
|
||||
quota.record(90)
|
||||
assertFalse(quota.isExceeded)
|
||||
assertEquals(0, quota.throttleTime)
|
||||
|
||||
// Recording a third value at T is rejected immediately and rate is not updated
|
||||
// because the quota is exhausted.
|
||||
assertThrows(classOf[ThrottlingQuotaExceededException],
|
||||
() => quota.record(90))
|
||||
assertTrue(quota.isExceeded)
|
||||
assertEquals(8000, quota.throttleTime)
|
||||
|
||||
// Throttle time is adjusted with time
|
||||
time.sleep(5000)
|
||||
assertEquals(3000, quota.throttleTime)
|
||||
|
||||
metrics.close()
|
||||
}
|
||||
}
|
||||
|
||||
class PermissiveControllerMutationQuotaTest {
|
||||
@Test
|
||||
def testControllerMutationQuotaViolation(): Unit = {
|
||||
val time = new MockTime(0, System.currentTimeMillis, 0)
|
||||
val metrics = new Metrics(time)
|
||||
val sensor = metrics.sensor("sensor", new MetricConfig()
|
||||
.quota(Quota.upperBound(10))
|
||||
.timeWindow(1, TimeUnit.SECONDS)
|
||||
.samples(11))
|
||||
val metricName = metrics.metricName("rate", "test-group")
|
||||
assertTrue(sensor.add(metricName, new Rate))
|
||||
|
||||
val quota = new PermissiveControllerMutationQuota(time, sensor)
|
||||
assertFalse(quota.isExceeded)
|
||||
|
||||
// Recording a first value at T to bring the avg rate to 9. Value is accepted
|
||||
// because the quota is not exhausted yet.
|
||||
quota.record(90)
|
||||
assertFalse(quota.isExceeded)
|
||||
assertEquals(0, quota.throttleTime)
|
||||
|
||||
// Recording a second value at T to bring the avg rate to 18. Value is accepted
|
||||
quota.record(90)
|
||||
assertFalse(quota.isExceeded)
|
||||
assertEquals(8000, quota.throttleTime)
|
||||
|
||||
// Recording a second value at T to bring the avg rate to 27. Value is accepted
|
||||
// and rate is updated even though the quota is exhausted.
|
||||
quota.record(90)
|
||||
assertFalse(quota.isExceeded) // quota is never exceeded
|
||||
assertEquals(17000, quota.throttleTime)
|
||||
|
||||
// Throttle time is adjusted with time
|
||||
time.sleep(5000)
|
||||
assertEquals(12000, quota.throttleTime)
|
||||
|
||||
metrics.close()
|
||||
}
|
||||
}
|
||||
|
||||
class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest {
|
||||
private val User = "ANONYMOUS"
|
||||
private val ClientId = "test-client"
|
||||
|
||||
private val config = ClientQuotaManagerConfig()
|
||||
|
||||
private def withQuotaManager(f: ControllerMutationQuotaManager => Unit): Unit = {
|
||||
val quotaManager = new ControllerMutationQuotaManager(config, metrics, time,"", None)
|
||||
try {
|
||||
f(quotaManager)
|
||||
} finally {
|
||||
quotaManager.shutdown()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testControllerMutationQuotaViolation(): Unit = {
|
||||
withQuotaManager { quotaManager =>
|
||||
quotaManager.updateQuota(Some(User), Some(ClientId), Some(ClientId),
|
||||
Some(Quota.upperBound(10)))
|
||||
val queueSizeMetric = metrics.metrics().get(
|
||||
metrics.metricName("queue-size", ControllerMutation.toString, ""))
|
||||
|
||||
// Verify that there is no quota violation if we remain under the quota.
|
||||
for (_ <- 0 until 10) {
|
||||
assertEquals(0, maybeRecord(quotaManager, User, ClientId, 10))
|
||||
time.sleep(1000)
|
||||
}
|
||||
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
|
||||
// Create a spike worth of 110 mutations.
|
||||
// Current avg rate = 10 * 10 = 100/10 = 10 mutations per second.
|
||||
// As we use the Strict enforcement, the quota is checked before updating the rate. Hence,
|
||||
// the spike is accepted and no quota violation error is raised.
|
||||
var throttleTime = maybeRecord(quotaManager, User, ClientId, 110)
|
||||
assertEquals("Should not be throttled", 0, throttleTime)
|
||||
|
||||
// Create a spike worth of 110 mutations.
|
||||
// Current avg rate = 10 * 10 + 110 = 210/10 = 21 mutations per second.
|
||||
// As the quota is already violated, the spike is rejected immediately without updating the
|
||||
// rate. The client must wait:
|
||||
// (21 - quota) / quota * window-size = (21 - 10) / 10 * 10 = 11 seconds
|
||||
throttleTime = maybeRecord(quotaManager, User, ClientId, 110)
|
||||
assertEquals("Should be throttled", 11000, throttleTime)
|
||||
|
||||
// Throttle
|
||||
throttle(quotaManager, User, ClientId, throttleTime, callback)
|
||||
assertEquals(1, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
|
||||
// After a request is delayed, the callback cannot be triggered immediately
|
||||
quotaManager.throttledChannelReaper.doWork()
|
||||
assertEquals(0, numCallbacks)
|
||||
|
||||
// Callback can only be triggered after the delay time passes
|
||||
time.sleep(throttleTime)
|
||||
quotaManager.throttledChannelReaper.doWork()
|
||||
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
|
||||
assertEquals(1, numCallbacks)
|
||||
|
||||
// Retry to spike worth of 110 mutations after having waited the required throttle time.
|
||||
// Current avg rate = 0 = 0/11 = 0 mutations per second.
|
||||
throttleTime = maybeRecord(quotaManager, User, ClientId, 110)
|
||||
assertEquals("Should be throttled", 0, throttleTime)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testNewStrictQuotaForReturnsUnboundedQuotaWhenQuotaIsDisabled(): Unit = {
|
||||
withQuotaManager { quotaManager =>
|
||||
assertEquals(UnboundedControllerMutationQuota,
|
||||
quotaManager.newStrictQuotaFor(buildSession(User), ClientId))
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testNewStrictQuotaForReturnsStrictQuotaWhenQuotaIsEnabled(): Unit = {
|
||||
withQuotaManager { quotaManager =>
|
||||
quotaManager.updateQuota(Some(User), Some(ClientId), Some(ClientId),
|
||||
Some(Quota.upperBound(10)))
|
||||
val quota = quotaManager.newStrictQuotaFor(buildSession(User), ClientId)
|
||||
assertTrue(quota.isInstanceOf[StrictControllerMutationQuota])
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testNewPermissiveQuotaForReturnsUnboundedQuotaWhenQuotaIsDisabled(): Unit = {
|
||||
withQuotaManager { quotaManager =>
|
||||
assertEquals(UnboundedControllerMutationQuota,
|
||||
quotaManager.newPermissiveQuotaFor(buildSession(User), ClientId))
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testNewPermissiveQuotaForReturnsStrictQuotaWhenQuotaIsEnabled(): Unit = {
|
||||
withQuotaManager { quotaManager =>
|
||||
quotaManager.updateQuota(Some(User), Some(ClientId), Some(ClientId),
|
||||
Some(Quota.upperBound(10)))
|
||||
val quota = quotaManager.newPermissiveQuotaFor(buildSession(User), ClientId)
|
||||
assertTrue(quota.isInstanceOf[PermissiveControllerMutationQuota])
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,366 @@
|
|||
/**
|
||||
* Licensed 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 java.util.Properties
|
||||
import java.util.concurrent.ExecutionException
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.internals.KafkaFutureImpl
|
||||
import org.apache.kafka.common.message.CreatePartitionsRequestData
|
||||
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
|
||||
import org.apache.kafka.common.message.CreateTopicsRequestData
|
||||
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic
|
||||
import org.apache.kafka.common.message.DeleteTopicsRequestData
|
||||
import org.apache.kafka.common.protocol.ApiKeys
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
import org.apache.kafka.common.quota.ClientQuotaAlteration
|
||||
import org.apache.kafka.common.quota.ClientQuotaEntity
|
||||
import org.apache.kafka.common.requests.AlterClientQuotasRequest
|
||||
import org.apache.kafka.common.requests.AlterClientQuotasResponse
|
||||
import org.apache.kafka.common.requests.CreatePartitionsRequest
|
||||
import org.apache.kafka.common.requests.CreatePartitionsResponse
|
||||
import org.apache.kafka.common.requests.CreateTopicsRequest
|
||||
import org.apache.kafka.common.requests.CreateTopicsResponse
|
||||
import org.apache.kafka.common.requests.DeleteTopicsRequest
|
||||
import org.apache.kafka.common.requests.DeleteTopicsResponse
|
||||
import org.apache.kafka.common.security.auth.AuthenticationContext
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder
|
||||
import org.junit.Assert.assertEquals
|
||||
import org.junit.Assert.assertTrue
|
||||
import org.junit.Before
|
||||
import org.junit.Test
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
object ControllerMutationQuotaTest {
|
||||
// Principal used for all client connections. This is updated by each test.
|
||||
var principal = KafkaPrincipal.ANONYMOUS
|
||||
class TestPrincipalBuilder extends KafkaPrincipalBuilder {
|
||||
override def build(context: AuthenticationContext): KafkaPrincipal = {
|
||||
principal
|
||||
}
|
||||
}
|
||||
|
||||
def asPrincipal(newPrincipal: KafkaPrincipal)(f: => Unit): Unit = {
|
||||
val currentPrincipal = principal
|
||||
principal = newPrincipal
|
||||
try f
|
||||
finally principal = currentPrincipal
|
||||
}
|
||||
|
||||
val ThrottledPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "ThrottledPrincipal")
|
||||
val UnboundedPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "UnboundedPrincipal")
|
||||
|
||||
val StrictCreateTopicsRequestVersion = ApiKeys.CREATE_TOPICS.latestVersion
|
||||
val PermissiveCreateTopicsRequestVersion = 5.toShort
|
||||
|
||||
val StrictDeleteTopicsRequestVersion = ApiKeys.DELETE_TOPICS.latestVersion
|
||||
val PermissiveDeleteTopicsRequestVersion = 4.toShort
|
||||
|
||||
val StrictCreatePartitionsRequestVersion = ApiKeys.CREATE_PARTITIONS.latestVersion
|
||||
val PermissiveCreatePartitionsRequestVersion = 2.toShort
|
||||
|
||||
val Topic1 = "topic-1"
|
||||
val Topic2 = "topic-2"
|
||||
val TopicsWithOnePartition = Map(Topic1 -> 1, Topic2 -> 1)
|
||||
val TopicsWith30Partitions = Map(Topic1 -> 30, Topic2 -> 30)
|
||||
val TopicsWith31Partitions = Map(Topic1 -> 31, Topic2 -> 31)
|
||||
|
||||
val ControllerMutationRate = 2.0
|
||||
}
|
||||
|
||||
class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||
import ControllerMutationQuotaTest._
|
||||
|
||||
override def brokerCount: Int = 1
|
||||
|
||||
override def brokerPropertyOverrides(properties: Properties): Unit = {
|
||||
properties.put(KafkaConfig.ControlledShutdownEnableProp, "false")
|
||||
properties.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "1")
|
||||
properties.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
|
||||
properties.put(KafkaConfig.PrincipalBuilderClassProp,
|
||||
classOf[ControllerMutationQuotaTest.TestPrincipalBuilder].getName)
|
||||
// We use the default number of samples and window size.
|
||||
properties.put(KafkaConfig.NumControllerQuotaSamplesProp, "11")
|
||||
properties.put(KafkaConfig.ControllerQuotaWindowSizeSecondsProp, "1")
|
||||
}
|
||||
|
||||
@Before
|
||||
override def setUp(): Unit = {
|
||||
super.setUp()
|
||||
|
||||
// Define a quota for ThrottledPrincipal
|
||||
defineUserQuota(ThrottledPrincipal.getName, Some(ControllerMutationRate))
|
||||
waitUserQuota(ThrottledPrincipal.getName, ControllerMutationRate)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testSetUnsetQuota(): Unit = {
|
||||
val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "User")
|
||||
// Default Value
|
||||
waitUserQuota(principal.getName, Long.MaxValue)
|
||||
// Define a new quota
|
||||
defineUserQuota(principal.getName, Some(ControllerMutationRate))
|
||||
// Check it
|
||||
waitUserQuota(principal.getName, ControllerMutationRate)
|
||||
// Remove it
|
||||
defineUserQuota(principal.getName, None)
|
||||
// Back to the default
|
||||
waitUserQuota(principal.getName, Long.MaxValue)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStrictCreateTopicsRequest(): Unit = {
|
||||
asPrincipal(ThrottledPrincipal) {
|
||||
// Create two topics worth of 30 partitions each. As we use a strict quota, we
|
||||
// expect one to be created and one to be rejected.
|
||||
// Theoretically, the throttle time should be below or equal to:
|
||||
// ((30 / 10) - 2) / 2 * 10 = 5s
|
||||
val (throttleTimeMs1, errors1) = createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
||||
assertThrottleTime(5000, throttleTimeMs1)
|
||||
// Ordering is not guaranteed so we only check the errors
|
||||
assertEquals(Set(Errors.NONE, Errors.THROTTLING_QUOTA_EXCEEDED), errors1.values.toSet)
|
||||
|
||||
// Retry the rejected topic. It should succeed after the throttling delay is passed and the
|
||||
// throttle time should be zero.
|
||||
val rejectedTopicName = errors1.filter(_._2 == Errors.THROTTLING_QUOTA_EXCEEDED).keys.head
|
||||
val rejectedTopicSpec = TopicsWith30Partitions.filter(_._1 == rejectedTopicName)
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
val (throttleTimeMs2, errors2) = createTopics(rejectedTopicSpec, StrictCreateTopicsRequestVersion)
|
||||
throttleTimeMs2 == 0 && errors2 == Map(rejectedTopicName -> Errors.NONE)
|
||||
}, "Failed to create topics after having been throttled")
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPermissiveCreateTopicsRequest(): Unit = {
|
||||
asPrincipal(ThrottledPrincipal) {
|
||||
// Create two topics worth of 30 partitions each. As we use a permissive quota, we
|
||||
// expect both topics to be created.
|
||||
// Theoretically, the throttle time should be below or equal to:
|
||||
// ((60 / 10) - 2) / 2 * 10 = 20s
|
||||
val (throttleTimeMs, errors) = createTopics(TopicsWith30Partitions, PermissiveCreateTopicsRequestVersion)
|
||||
assertThrottleTime(20000, throttleTimeMs)
|
||||
assertEquals(Map(Topic1 -> Errors.NONE, Topic2 -> Errors.NONE), errors)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testUnboundedCreateTopicsRequest(): Unit = {
|
||||
asPrincipal(UnboundedPrincipal) {
|
||||
// Create two topics worth of 30 partitions each. As we use an user without quota, we
|
||||
// expect both topics to be created. The throttle time should be equal to 0.
|
||||
val (throttleTimeMs, errors) = createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
||||
assertEquals(0, throttleTimeMs)
|
||||
assertEquals(Map(Topic1 -> Errors.NONE, Topic2 -> Errors.NONE), errors)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStrictDeleteTopicsRequest(): Unit = {
|
||||
asPrincipal(UnboundedPrincipal) {
|
||||
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
||||
}
|
||||
|
||||
asPrincipal(ThrottledPrincipal) {
|
||||
// Delete two topics worth of 30 partitions each. As we use a strict quota, we
|
||||
// expect the first topic to be deleted and the second to be rejected.
|
||||
// Theoretically, the throttle time should be below or equal to:
|
||||
// ((30 / 10) - 2) / 2 * 10 = 5s
|
||||
val (throttleTimeMs1, errors1) = deleteTopics(TopicsWith30Partitions, StrictDeleteTopicsRequestVersion)
|
||||
assertThrottleTime(5000, throttleTimeMs1)
|
||||
// Ordering is not guaranteed so we only check the errors
|
||||
assertEquals(Set(Errors.NONE, Errors.THROTTLING_QUOTA_EXCEEDED), errors1.values.toSet)
|
||||
|
||||
// Retry the rejected topic. It should succeed after the throttling delay is passed and the
|
||||
// throttle time should be zero.
|
||||
val rejectedTopicName = errors1.filter(_._2 == Errors.THROTTLING_QUOTA_EXCEEDED).keys.head
|
||||
val rejectedTopicSpec = TopicsWith30Partitions.filter(_._1 == rejectedTopicName)
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
val (throttleTimeMs2, errors2) = deleteTopics(rejectedTopicSpec, StrictDeleteTopicsRequestVersion)
|
||||
throttleTimeMs2 == 0 && errors2 == Map(rejectedTopicName -> Errors.NONE)
|
||||
}, "Failed to delete topics after having been throttled")
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPermissiveDeleteTopicsRequest(): Unit = {
|
||||
asPrincipal(UnboundedPrincipal) {
|
||||
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
||||
}
|
||||
|
||||
asPrincipal(ThrottledPrincipal) {
|
||||
// Delete two topics worth of 30 partitions each. As we use a permissive quota, we
|
||||
// expect both topics to be deleted.
|
||||
// Theoretically, the throttle time should be below or equal to:
|
||||
// ((60 / 10) - 2) / 2 * 10 = 20s
|
||||
val (throttleTimeMs, errors) = deleteTopics(TopicsWith30Partitions, PermissiveDeleteTopicsRequestVersion)
|
||||
assertThrottleTime(20000, throttleTimeMs)
|
||||
assertEquals(Map(Topic1 -> Errors.NONE, Topic2 -> Errors.NONE), errors)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testUnboundedDeleteTopicsRequest(): Unit = {
|
||||
asPrincipal(UnboundedPrincipal) {
|
||||
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
||||
|
||||
// Delete two topics worth of 30 partitions each. As we use an user without quota, we
|
||||
// expect both topics to be deleted. The throttle time should be equal to 0.
|
||||
val (throttleTimeMs, errors) = deleteTopics(TopicsWith30Partitions, StrictDeleteTopicsRequestVersion)
|
||||
assertEquals(0, throttleTimeMs)
|
||||
assertEquals(Map(Topic1 -> Errors.NONE, Topic2 -> Errors.NONE), errors)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testStrictCreatePartitionsRequest(): Unit = {
|
||||
asPrincipal(UnboundedPrincipal) {
|
||||
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
||||
}
|
||||
|
||||
asPrincipal(ThrottledPrincipal) {
|
||||
// Add 30 partitions to each topic. As we use a strict quota, we
|
||||
// expect the first topic to be extended and the second to be rejected.
|
||||
// Theoretically, the throttle time should be below or equal to:
|
||||
// ((30 / 10) - 2) / 2 * 10 = 5s
|
||||
val (throttleTimeMs1, errors1) = createPartitions(TopicsWith31Partitions, StrictCreatePartitionsRequestVersion)
|
||||
assertThrottleTime(5000, throttleTimeMs1)
|
||||
// Ordering is not guaranteed so we only check the errors
|
||||
assertEquals(Set(Errors.NONE, Errors.THROTTLING_QUOTA_EXCEEDED), errors1.values.toSet)
|
||||
|
||||
// Retry the rejected topic. It should succeed after the throttling delay is passed and the
|
||||
// throttle time should be zero.
|
||||
val rejectedTopicName = errors1.filter(_._2 == Errors.THROTTLING_QUOTA_EXCEEDED).keys.head
|
||||
val rejectedTopicSpec = TopicsWith30Partitions.filter(_._1 == rejectedTopicName)
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
val (throttleTimeMs2, errors2) = createPartitions(rejectedTopicSpec, StrictCreatePartitionsRequestVersion)
|
||||
throttleTimeMs2 == 0 && errors2 == Map(rejectedTopicName -> Errors.NONE)
|
||||
}, "Failed to create partitions after having been throttled")
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPermissiveCreatePartitionsRequest(): Unit = {
|
||||
asPrincipal(UnboundedPrincipal) {
|
||||
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
||||
}
|
||||
|
||||
asPrincipal(ThrottledPrincipal) {
|
||||
// Create two topics worth of 30 partitions each. As we use a permissive quota, we
|
||||
// expect both topics to be created.
|
||||
// Theoretically, the throttle time should be below or equal to:
|
||||
// ((60 / 10) - 2) / 2 * 10 = 20s
|
||||
val (throttleTimeMs, errors) = createPartitions(TopicsWith31Partitions, PermissiveCreatePartitionsRequestVersion)
|
||||
assertThrottleTime(20000, throttleTimeMs)
|
||||
assertEquals(Map(Topic1 -> Errors.NONE, Topic2 -> Errors.NONE), errors)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testUnboundedCreatePartitionsRequest(): Unit = {
|
||||
asPrincipal(UnboundedPrincipal) {
|
||||
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
||||
|
||||
// Create two topics worth of 30 partitions each. As we use an user without quota, we
|
||||
// expect both topics to be created. The throttle time should be equal to 0.
|
||||
val (throttleTimeMs, errors) = createPartitions(TopicsWith31Partitions, StrictCreatePartitionsRequestVersion)
|
||||
assertEquals(0, throttleTimeMs)
|
||||
assertEquals(Map(Topic1 -> Errors.NONE, Topic2 -> Errors.NONE), errors)
|
||||
}
|
||||
}
|
||||
|
||||
private def assertThrottleTime(max: Int, actual: Int): Unit = {
|
||||
assertTrue(
|
||||
s"Expected a throttle time between 0 and $max but got $actual",
|
||||
(actual >= 0) && (actual <= max))
|
||||
}
|
||||
|
||||
private def createTopics(topics: Map[String, Int], version: Short): (Int, Map[String, Errors]) = {
|
||||
val data = new CreateTopicsRequestData()
|
||||
topics.foreach { case (topic, numPartitions) =>
|
||||
data.topics.add(new CreatableTopic()
|
||||
.setName(topic).setNumPartitions(numPartitions).setReplicationFactor(1))
|
||||
}
|
||||
val request = new CreateTopicsRequest.Builder(data).build(version)
|
||||
val response = connectAndReceive[CreateTopicsResponse](request)
|
||||
response.data.throttleTimeMs -> response.data.topics.asScala
|
||||
.map(topic => topic.name -> Errors.forCode(topic.errorCode)).toMap
|
||||
}
|
||||
|
||||
private def deleteTopics(topics: Map[String, Int], version: Short): (Int, Map[String, Errors]) = {
|
||||
val data = new DeleteTopicsRequestData()
|
||||
.setTimeoutMs(60000)
|
||||
.setTopicNames(topics.keys.toSeq.asJava)
|
||||
val request = new DeleteTopicsRequest.Builder(data).build(version)
|
||||
val response = connectAndReceive[DeleteTopicsResponse](request)
|
||||
response.data.throttleTimeMs -> response.data.responses.asScala
|
||||
.map(topic => topic.name -> Errors.forCode(topic.errorCode)).toMap
|
||||
}
|
||||
|
||||
private def createPartitions(topics: Map[String, Int], version: Short): (Int, Map[String, Errors]) = {
|
||||
val data = new CreatePartitionsRequestData().setTimeoutMs(60000)
|
||||
topics.foreach { case (topic, numPartitions) =>
|
||||
data.topics.add(new CreatePartitionsTopic()
|
||||
.setName(topic).setCount(numPartitions).setAssignments(null))
|
||||
}
|
||||
val request = new CreatePartitionsRequest.Builder(data).build(version)
|
||||
val response = connectAndReceive[CreatePartitionsResponse](request)
|
||||
response.data.throttleTimeMs -> response.data.results.asScala
|
||||
.map(topic => topic.name -> Errors.forCode(topic.errorCode)).toMap
|
||||
}
|
||||
|
||||
private def defineUserQuota(user: String, quota: Option[Double]): Unit = {
|
||||
val entity = new ClientQuotaEntity(Map(ClientQuotaEntity.USER -> user).asJava)
|
||||
val quotas = Map(DynamicConfig.Client.ControllerMutationOverrideProp -> quota)
|
||||
|
||||
try alterClientQuotas(Map(entity -> quotas))(entity).get(10, TimeUnit.SECONDS) catch {
|
||||
case e: ExecutionException => throw e.getCause
|
||||
}
|
||||
}
|
||||
|
||||
private def waitUserQuota(user: String, expectedQuota: Double): Unit = {
|
||||
val quotaManager = servers.head.quotaManagers.controllerMutation
|
||||
var actualQuota = Double.MinValue
|
||||
|
||||
TestUtils.waitUntilTrue(() => {
|
||||
actualQuota = quotaManager.quota(user, "").bound()
|
||||
expectedQuota == actualQuota
|
||||
}, s"Quota of $user is not $expectedQuota but $actualQuota")
|
||||
}
|
||||
|
||||
private def alterClientQuotas(request: Map[ClientQuotaEntity, Map[String, Option[Double]]]): Map[ClientQuotaEntity, KafkaFutureImpl[Void]] = {
|
||||
val entries = request.map { case (entity, alter) =>
|
||||
val ops = alter.map { case (key, value) =>
|
||||
new ClientQuotaAlteration.Op(key, value.map(Double.box).orNull)
|
||||
}.asJavaCollection
|
||||
new ClientQuotaAlteration(entity, ops)
|
||||
}
|
||||
|
||||
val response = request.map(e => e._1 -> new KafkaFutureImpl[Void]).asJava
|
||||
sendAlterClientQuotasRequest(entries).complete(response)
|
||||
val result = response.asScala
|
||||
assertEquals(request.size, result.size)
|
||||
request.foreach(e => assertTrue(result.get(e._1).isDefined))
|
||||
result.toMap
|
||||
}
|
||||
|
||||
private def sendAlterClientQuotasRequest(entries: Iterable[ClientQuotaAlteration]): AlterClientQuotasResponse = {
|
||||
val request = new AlterClientQuotasRequest.Builder(entries.asJavaCollection, false).build()
|
||||
connectAndReceive[AlterClientQuotasResponse](request, destination = controllerSocketServer)
|
||||
}
|
||||
}
|
|
@ -94,9 +94,10 @@ class KafkaApisTest {
|
|||
private val metadataCache = new MetadataCache(brokerId)
|
||||
private val clientQuotaManager: ClientQuotaManager = EasyMock.createNiceMock(classOf[ClientQuotaManager])
|
||||
private val clientRequestQuotaManager: ClientRequestQuotaManager = EasyMock.createNiceMock(classOf[ClientRequestQuotaManager])
|
||||
private val clientControllerQuotaManager: ControllerMutationQuotaManager = EasyMock.createNiceMock(classOf[ControllerMutationQuotaManager])
|
||||
private val replicaQuotaManager: ReplicationQuotaManager = EasyMock.createNiceMock(classOf[ReplicationQuotaManager])
|
||||
private val quotas = QuotaManagers(clientQuotaManager, clientQuotaManager, clientRequestQuotaManager,
|
||||
replicaQuotaManager, replicaQuotaManager, replicaQuotaManager, None)
|
||||
clientControllerQuotaManager, replicaQuotaManager, replicaQuotaManager, replicaQuotaManager, None)
|
||||
private val fetchManager: FetchManager = EasyMock.createNiceMock(classOf[FetchManager])
|
||||
private val brokerTopicStats = new BrokerTopicStats
|
||||
private val clusterId = "clusterId"
|
||||
|
|
|
@ -177,22 +177,16 @@ class RequestQuotaTest extends BaseRequestTest {
|
|||
}
|
||||
|
||||
private def throttleTimeMetricValueForQuotaType(clientId: String, quotaType: QuotaType): Double = {
|
||||
val metricName = leaderNode.metrics.metricName("throttle-time",
|
||||
quotaType.toString,
|
||||
"",
|
||||
"user", "",
|
||||
"client-id", clientId)
|
||||
val metricName = leaderNode.metrics.metricName("throttle-time", quotaType.toString,
|
||||
"", "user", "", "client-id", clientId)
|
||||
val sensor = leaderNode.quotaManagers.request.getOrCreateQuotaSensors(session("ANONYMOUS"),
|
||||
clientId).throttleTimeSensor
|
||||
metricValue(leaderNode.metrics.metrics.get(metricName), sensor)
|
||||
}
|
||||
|
||||
private def requestTimeMetricValue(clientId: String): Double = {
|
||||
val metricName = leaderNode.metrics.metricName("request-time",
|
||||
QuotaType.Request.toString,
|
||||
"",
|
||||
"user", "",
|
||||
"client-id", clientId)
|
||||
val metricName = leaderNode.metrics.metricName("request-time", QuotaType.Request.toString,
|
||||
"", "user", "", "client-id", clientId)
|
||||
val sensor = leaderNode.quotaManagers.request.getOrCreateQuotaSensors(session("ANONYMOUS"),
|
||||
clientId).quotaSensor
|
||||
metricValue(leaderNode.metrics.metrics.get(metricName), sensor)
|
||||
|
|
|
@ -827,7 +827,7 @@ object TestUtils extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Wait until the given condition is true or throw an exception if the given wait time elapses.
|
||||
* Wait until the given condition is true or throw an exception if the given wait time elapses.
|
||||
*
|
||||
* @param condition condition to check
|
||||
* @param msg error message
|
||||
|
@ -857,7 +857,7 @@ object TestUtils extends Logging {
|
|||
* This method is useful in cases where `waitUntilTrue` makes it awkward to provide good error messages.
|
||||
*/
|
||||
def computeUntilTrue[T](compute: => T, waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, pause: Long = 100L)(
|
||||
predicate: T => Boolean): (T, Boolean) = {
|
||||
predicate: T => Boolean): (T, Boolean) = {
|
||||
val startTime = System.currentTimeMillis()
|
||||
while (true) {
|
||||
val result = compute
|
||||
|
|
|
@ -25,6 +25,7 @@ import kafka.server.AdminManager;
|
|||
import kafka.server.BrokerTopicStats;
|
||||
import kafka.server.ClientQuotaManager;
|
||||
import kafka.server.ClientRequestQuotaManager;
|
||||
import kafka.server.ControllerMutationQuotaManager;
|
||||
import kafka.server.FetchManager;
|
||||
import kafka.server.KafkaApis;
|
||||
import kafka.server.KafkaConfig;
|
||||
|
@ -100,10 +101,11 @@ public class MetadataRequestBenchmark {
|
|||
private MetadataCache metadataCache = new MetadataCache(brokerId);
|
||||
private ClientQuotaManager clientQuotaManager = Mockito.mock(ClientQuotaManager.class);
|
||||
private ClientRequestQuotaManager clientRequestQuotaManager = Mockito.mock(ClientRequestQuotaManager.class);
|
||||
private ControllerMutationQuotaManager controllerMutationQuotaManager = Mockito.mock(ControllerMutationQuotaManager.class);
|
||||
private ReplicationQuotaManager replicaQuotaManager = Mockito.mock(ReplicationQuotaManager.class);
|
||||
private QuotaFactory.QuotaManagers quotaManagers = new QuotaFactory.QuotaManagers(clientQuotaManager,
|
||||
clientQuotaManager, clientRequestQuotaManager, replicaQuotaManager, replicaQuotaManager,
|
||||
replicaQuotaManager, Option.empty());
|
||||
clientQuotaManager, clientRequestQuotaManager, controllerMutationQuotaManager, replicaQuotaManager,
|
||||
replicaQuotaManager, replicaQuotaManager, Option.empty());
|
||||
private FetchManager fetchManager = Mockito.mock(FetchManager.class);
|
||||
private BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
|
||||
private KafkaPrincipal principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "test-user");
|
||||
|
|
Loading…
Reference in New Issue