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:
David Jacot 2020-07-22 17:38:55 +02:00 committed by GitHub
parent 194c56fce2
commit a5ffd1ca44
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 1769 additions and 593 deletions

View File

@ -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;
}
}

View File

@ -97,7 +97,6 @@ public final class Sensor {
public boolean shouldRecord(final int configId) { public boolean shouldRecord(final int configId) {
return configId == DEBUG.id || configId == this.id; return configId == DEBUG.id || configId == this.id;
} }
} }
private final RecordingLevel recordingLevel; 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) { public void record(double value, long timeMs, boolean checkQuotas) {
if (shouldRecord()) { if (shouldRecord()) {
recordInternal(value, timeMs, checkQuotas); recordInternal(value, timeMs, checkQuotas);

View File

@ -78,6 +78,7 @@ import org.apache.kafka.common.errors.OffsetNotAvailableException;
import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.errors.OffsetOutOfRangeException;
import org.apache.kafka.common.errors.OperationNotAttemptedException; import org.apache.kafka.common.errors.OperationNotAttemptedException;
import org.apache.kafka.common.errors.OutOfOrderSequenceException; 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.UnstableOffsetCommitException;
import org.apache.kafka.common.errors.PolicyViolationException; import org.apache.kafka.common.errors.PolicyViolationException;
import org.apache.kafka.common.errors.PreferredLeaderNotAvailableException; 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.", GROUP_SUBSCRIBED_TO_TOPIC(86, "Deleting offsets of a topic is forbidden while the consumer group is actively subscribed to it.",
GroupSubscribedToTopicException::new), GroupSubscribedToTopicException::new),
INVALID_RECORD(87, "This record has failed the validation on broker and hence will be rejected.", InvalidRecordException::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); private static final Logger log = LoggerFactory.getLogger(Errors.class);

View File

@ -22,5 +22,6 @@ package org.apache.kafka.server.quota;
public enum ClientQuotaType { public enum ClientQuotaType {
PRODUCE, PRODUCE,
FETCH, FETCH,
REQUEST REQUEST,
CONTROLLER_MUTATION
} }

View File

@ -18,8 +18,12 @@
"type": "request", "type": "request",
"name": "CreatePartitionsRequest", "name": "CreatePartitionsRequest",
// Version 1 is the same as version 0. // Version 1 is the same as version 0.
//
// Version 2 adds flexible version support // 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+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "Topics", "type": "[]CreatePartitionsTopic", "versions": "0+", { "name": "Topics", "type": "[]CreatePartitionsTopic", "versions": "0+",

View File

@ -18,8 +18,12 @@
"type": "response", "type": "response",
"name": "CreatePartitionsResponse", "name": "CreatePartitionsResponse",
// Starting in version 1, on quota violation, brokers send out responses before throttling. // Starting in version 1, on quota violation, brokers send out responses before throttling.
//
// Version 2 adds flexible version support // 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+", "flexibleVersions": "2+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",

View File

@ -23,7 +23,10 @@
// //
// Version 5 is the first flexible version. // Version 5 is the first flexible version.
// Version 5 also returns topic configs in the response (KIP-525). // 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+", "flexibleVersions": "5+",
"fields": [ "fields": [
{ "name": "Topics", "type": "[]CreatableTopic", "versions": "0+", { "name": "Topics", "type": "[]CreatableTopic", "versions": "0+",

View File

@ -27,7 +27,10 @@
// //
// Version 5 is the first flexible version. // Version 5 is the first flexible version.
// Version 5 also returns topic configs in the response (KIP-525). // 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+", "flexibleVersions": "5+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "2+", "ignorable": true,

View File

@ -20,7 +20,10 @@
// Versions 0, 1, 2, and 3 are the same. // Versions 0, 1, 2, and 3 are the same.
// //
// Version 4 is the first flexible version. // 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+", "flexibleVersions": "4+",
"fields": [ "fields": [
{ "name": "TopicNames", "type": "[]string", "versions": "0+", "entityType": "topicName", { "name": "TopicNames", "type": "[]string", "versions": "0+", "entityType": "topicName",

View File

@ -24,7 +24,10 @@
// Starting in version 3, a TOPIC_DELETION_DISABLED error code may be returned. // Starting in version 3, a TOPIC_DELETION_DISABLED error code may be returned.
// //
// Version 4 is the first flexible version. // 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+", "flexibleVersions": "4+",
"fields": [ "fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
@ -34,7 +37,9 @@
{ "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName", { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName",
"about": "The topic name" }, "about": "The topic name" },
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "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." }
]} ]}
] ]
} }

View File

@ -28,7 +28,6 @@ import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.config.ConfigResource; 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.NotCoordinatorException;
import org.apache.kafka.common.errors.NotEnoughReplicasException; import org.apache.kafka.common.errors.NotEnoughReplicasException;
import org.apache.kafka.common.errors.SecurityDisabledException; 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, 2, true);
checkResponse(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE, 3, true); checkResponse(ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE, 3, true);
checkRequest(createCreateTopicRequest(0), true); for (int v = ApiKeys.CREATE_TOPICS.oldestVersion(); v <= ApiKeys.CREATE_TOPICS.latestVersion(); v++) {
checkErrorResponse(createCreateTopicRequest(0), unknownServerException, true); checkRequest(createCreateTopicRequest(v), true);
checkResponse(createCreateTopicResponse(), 0, true); checkErrorResponse(createCreateTopicRequest(v), unknownServerException, true);
checkRequest(createCreateTopicRequest(1), true); checkResponse(createCreateTopicResponse(), v, 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);
checkRequest(createDeleteTopicsRequest(), true); for (int v = ApiKeys.DELETE_TOPICS.oldestVersion(); v <= ApiKeys.DELETE_TOPICS.latestVersion(); v++) {
checkErrorResponse(createDeleteTopicsRequest(), unknownServerException, true); checkRequest(createDeleteTopicsRequest(v), true);
checkResponse(createDeleteTopicsResponse(), 0, 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); checkRequest(createInitPidRequest(), true);
checkErrorResponse(createInitPidRequest(), unknownServerException, true); checkErrorResponse(createInitPidRequest(), unknownServerException, true);
@ -454,10 +449,6 @@ public class RequestResponseTest {
checkErrorResponse(createDescribeConfigsRequest(1), unknownServerException, true); checkErrorResponse(createDescribeConfigsRequest(1), unknownServerException, true);
checkResponse(createDescribeConfigsResponse((short) 1), 1, false); checkResponse(createDescribeConfigsResponse((short) 1), 1, false);
checkDescribeConfigsResponseVersions(); checkDescribeConfigsResponseVersions();
checkRequest(createCreatePartitionsRequest(), true);
checkRequest(createCreatePartitionsRequestWithAssignments(), false);
checkErrorResponse(createCreatePartitionsRequest(), new InvalidTopicException(), true);
checkResponse(createCreatePartitionsResponse(), 0, true);
checkRequest(createCreateTokenRequest(), true); checkRequest(createCreateTokenRequest(), true);
checkErrorResponse(createCreateTokenRequest(), unknownServerException, true); checkErrorResponse(createCreateTokenRequest(), unknownServerException, true);
checkResponse(createCreateTokenResponse(), 0, true); checkResponse(createCreateTokenResponse(), 0, true);
@ -1602,23 +1593,23 @@ public class RequestResponseTest {
} }
private CreateTopicsRequest createCreateTopicRequest(int version, boolean validateOnly) { private CreateTopicsRequest createCreateTopicRequest(int version, boolean validateOnly) {
CreateTopicsRequestData data = new CreateTopicsRequestData(). CreateTopicsRequestData data = new CreateTopicsRequestData()
setTimeoutMs(123). .setTimeoutMs(123)
setValidateOnly(validateOnly); .setValidateOnly(validateOnly);
data.topics().add(new CreatableTopic(). data.topics().add(new CreatableTopic()
setNumPartitions(3). .setNumPartitions(3)
setReplicationFactor((short) 5)); .setReplicationFactor((short) 5));
CreatableTopic topic2 = new CreatableTopic(); CreatableTopic topic2 = new CreatableTopic();
data.topics().add(topic2); data.topics().add(topic2);
topic2.assignments().add(new CreatableReplicaAssignment(). topic2.assignments().add(new CreatableReplicaAssignment()
setPartitionIndex(0). .setPartitionIndex(0)
setBrokerIds(Arrays.asList(1, 2, 3))); .setBrokerIds(Arrays.asList(1, 2, 3)));
topic2.assignments().add(new CreatableReplicaAssignment(). topic2.assignments().add(new CreatableReplicaAssignment()
setPartitionIndex(1). .setPartitionIndex(1)
setBrokerIds(Arrays.asList(2, 3, 4))); .setBrokerIds(Arrays.asList(2, 3, 4)));
topic2.configs().add(new CreateableTopicConfig(). topic2.configs().add(new CreateableTopicConfig()
setName("config1").setValue("value1")); .setName("config1").setValue("value1"));
return new CreateTopicsRequest.Builder(data).build((short) version); return new CreateTopicsRequest.Builder(data).build((short) version);
} }
@ -1644,21 +1635,23 @@ public class RequestResponseTest {
return new CreateTopicsResponse(data); return new CreateTopicsResponse(data);
} }
private DeleteTopicsRequest createDeleteTopicsRequest() { private DeleteTopicsRequest createDeleteTopicsRequest(int version) {
return new DeleteTopicsRequest.Builder( return new DeleteTopicsRequest.Builder(new DeleteTopicsRequestData()
new DeleteTopicsRequestData() .setTopicNames(Arrays.asList("my_t1", "my_t2"))
.setTopicNames(Arrays.asList("my_t1", "my_t2")) .setTimeoutMs(1000)
.setTimeoutMs(1000)).build(); ).build((short) version);
} }
private DeleteTopicsResponse createDeleteTopicsResponse() { private DeleteTopicsResponse createDeleteTopicsResponse() {
DeleteTopicsResponseData data = new DeleteTopicsResponseData(); DeleteTopicsResponseData data = new DeleteTopicsResponseData();
data.responses().add(new DeletableTopicResult() data.responses().add(new DeletableTopicResult()
.setName("t1") .setName("t1")
.setErrorCode(Errors.INVALID_TOPIC_EXCEPTION.code())); .setErrorCode(Errors.INVALID_TOPIC_EXCEPTION.code())
.setErrorMessage("Error Message"));
data.responses().add(new DeletableTopicResult() data.responses().add(new DeletableTopicResult()
.setName("t2") .setName("t2")
.setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code())); .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code())
.setErrorMessage("Error Message"));
return new DeleteTopicsResponse(data); return new DeleteTopicsResponse(data);
} }
@ -2019,7 +2012,7 @@ public class RequestResponseTest {
return new AlterConfigsResponse(data); return new AlterConfigsResponse(data);
} }
private CreatePartitionsRequest createCreatePartitionsRequest() { private CreatePartitionsRequest createCreatePartitionsRequest(int version) {
List<CreatePartitionsTopic> topics = new LinkedList<>(); List<CreatePartitionsTopic> topics = new LinkedList<>();
topics.add(new CreatePartitionsTopic() topics.add(new CreatePartitionsTopic()
.setName("my_topic") .setName("my_topic")
@ -2034,10 +2027,11 @@ public class RequestResponseTest {
.setTimeoutMs(0) .setTimeoutMs(0)
.setValidateOnly(false) .setValidateOnly(false)
.setTopics(topics); .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<>(); List<CreatePartitionsTopic> topics = new LinkedList<>();
CreatePartitionsAssignment myTopicAssignment = new CreatePartitionsAssignment() CreatePartitionsAssignment myTopicAssignment = new CreatePartitionsAssignment()
.setBrokerIds(Collections.singletonList(2)); .setBrokerIds(Collections.singletonList(2));
@ -2060,7 +2054,8 @@ public class RequestResponseTest {
.setTimeoutMs(0) .setTimeoutMs(0)
.setValidateOnly(false) .setValidateOnly(false)
.setTopics(topics); .setTopics(topics);
return new CreatePartitionsRequest(data, (short) 0);
return new CreatePartitionsRequest(data, (short) version);
} }
private CreatePartitionsResponse createCreatePartitionsResponse() { private CreatePartitionsResponse createCreatePartitionsResponse() {

View File

@ -181,11 +181,11 @@ abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: Stri
def closeAllFetchers(): Unit = { def closeAllFetchers(): Unit = {
lock synchronized { lock synchronized {
for ( (_, fetcher) <- fetcherThreadMap) { for ((_, fetcher) <- fetcherThreadMap) {
fetcher.initiateShutdown() fetcher.initiateShutdown()
} }
for ( (_, fetcher) <- fetcherThreadMap) { for ((_, fetcher) <- fetcherThreadMap) {
fetcher.shutdown() fetcher.shutdown()
} }
fetcherThreadMap.clear() fetcherThreadMap.clear()

View File

@ -30,6 +30,7 @@ import org.apache.kafka.clients.admin.AlterConfigOp
import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin.AlterConfigOp.OpType
import org.apache.kafka.common.config.ConfigDef.ConfigKey import org.apache.kafka.common.config.ConfigDef.ConfigKey
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, LogLevelConfig} 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.errors.{ApiException, InvalidConfigurationException, InvalidPartitionsException, InvalidReplicaAssignmentException, InvalidRequestException, ReassignmentInProgressException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedVersionException}
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic 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.DescribeConfigsResponseData
import org.apache.kafka.common.message.DescribeConfigsRequestData.DescribeConfigsResource import org.apache.kafka.common.message.DescribeConfigsRequestData.DescribeConfigsResource
import org.apache.kafka.common.metrics.Metrics 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.{AlterConfigPolicy, CreateTopicPolicy}
import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata
import org.apache.kafka.common.protocol.Errors 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.") 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. * 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. * 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, def createTopics(timeout: Int,
validateOnly: Boolean, validateOnly: Boolean,
toCreate: Map[String, CreatableTopic], toCreate: Map[String, CreatableTopic],
includeConfigsAndMetatadata: Map[String, CreatableTopicResult], includeConfigsAndMetadata: Map[String, CreatableTopicResult],
controllerMutationQuota: ControllerMutationQuota,
responseCallback: Map[String, ApiError] => Unit): Unit = { responseCallback: Map[String, ApiError] => Unit): Unit = {
// 1. map over topics creating assignment and calling zookeeper // 1. map over topics creating assignment and calling zookeeper
@ -102,12 +149,6 @@ class AdminManager(val config: KafkaConfig,
if (nullConfigs.nonEmpty) if (nullConfigs.nonEmpty)
throw new InvalidRequestException(s"Null value not supported for topic configs : ${nullConfigs.mkString(",")}") 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) if ((topic.numPartitions != NO_NUM_PARTITIONS || topic.replicationFactor != NO_REPLICATION_FACTOR)
&& !topic.assignments().isEmpty) { && !topic.assignments().isEmpty) {
throw new InvalidRequestException("Both numPartitions or replicationFactor and replicasAssignments were set. " + 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) val resolvedReplicationFactor = if (topic.replicationFactor == NO_REPLICATION_FACTOR)
defaultReplicationFactor else topic.replicationFactor defaultReplicationFactor else topic.replicationFactor
val assignments = if (topic.assignments().isEmpty) { val assignments = if (topic.assignments.isEmpty) {
AdminUtils.assignReplicasToBrokers( AdminUtils.assignReplicasToBrokers(
brokers, resolvedNumPartitions, resolvedReplicationFactor) brokers, resolvedNumPartitions, resolvedReplicationFactor)
} else { } else {
@ -133,76 +174,47 @@ class AdminManager(val config: KafkaConfig,
} }
trace(s"Assignments for topic $topic are $assignments ") trace(s"Assignments for topic $topic are $assignments ")
createTopicPolicy match { val configs = new Properties()
case Some(policy) => topic.configs.forEach(entry => configs.setProperty(entry.name, entry.value))
adminZkClient.validateTopicCreate(topic.name, assignments, configs) adminZkClient.validateTopicCreate(topic.name, assignments, configs)
validateTopicCreatePolicy(topic, resolvedNumPartitions, resolvedReplicationFactor, assignments)
// Use `null` for unset fields in the public API // For responses with DescribeConfigs permission, populate metadata and configs. It is
val numPartitions: java.lang.Integer = // safe to populate it before creating the topic because the values are unset if the
if (topic.assignments().isEmpty) resolvedNumPartitions else null // creation fails.
val replicationFactor: java.lang.Short = maybePopulateMetadataAndConfigs(includeConfigsAndMetadata, topic.name, configs, assignments)
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))
if (!validateOnly) if (validateOnly) {
adminZkClient.createTopicWithAssignment(topic.name, configs, assignments) CreatePartitionsMetadata(topic.name, assignments.keySet)
} else {
case None => controllerMutationQuota.record(assignments.size)
if (validateOnly) adminZkClient.createTopicWithAssignment(topic.name, configs, assignments, validate = false)
adminZkClient.validateTopicCreate(topic.name, assignments, configs) CreatePartitionsMetadata(topic.name, assignments.keySet)
else
adminZkClient.createTopicWithAssignment(topic.name, configs, assignments)
} }
// 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 { } catch {
// Log client errors at a lower level than unexpected exceptions // Log client errors at a lower level than unexpected exceptions
case e: TopicExistsException => case e: TopicExistsException =>
debug(s"Topic creation failed since topic '${topic.name}' already exists.", e) 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 => case e: ApiException =>
info(s"Error processing create topic request $topic", e) info(s"Error processing create topic request $topic", e)
CreatePartitionsMetadata(topic.name, Set.empty, ApiError.fromThrowable(e)) CreatePartitionsMetadata(topic.name, e)
case e: ConfigException => case e: ConfigException =>
info(s"Error processing create topic request $topic", e) 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 => case e: Throwable =>
error(s"Error processing create topic request $topic", e) error(s"Error processing create topic request $topic", e)
CreatePartitionsMetadata(topic.name, Set.empty, ApiError.fromThrowable(e)) CreatePartitionsMetadata(topic.name, e)
}).toBuffer }).toBuffer
// 2. if timeout <= 0, validateOnly or no topics can proceed return immediately // 2. if timeout <= 0, validateOnly or no topics can proceed return immediately
if (timeout <= 0 || validateOnly || !metadata.exists(_.error.is(Errors.NONE))) { if (timeout <= 0 || validateOnly || !metadata.exists(_.error.is(Errors.NONE))) {
val results = metadata.map { createTopicMetadata => val results = metadata.map { createTopicMetadata =>
// ignore topics that already have errors // 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)) (createTopicMetadata.topic, new ApiError(Errors.REQUEST_TIMED_OUT, null))
} else { } else {
(createTopicMetadata.topic, createTopicMetadata.error) (createTopicMetadata.topic, createTopicMetadata.error)
@ -211,8 +223,9 @@ class AdminManager(val config: KafkaConfig,
responseCallback(results) responseCallback(results)
} else { } else {
// 3. else pass the assignments and errors to the delayed operation and set the keys // 3. else pass the assignments and errors to the delayed operation and set the keys
val delayedCreate = new DelayedCreatePartitions(timeout, metadata, this, responseCallback) val delayedCreate = new DelayedCreatePartitions(timeout, metadata, this,
val delayedCreateKeys = toCreate.values.map(topic => new TopicKey(topic.name)).toBuffer responseCallback)
val delayedCreateKeys = toCreate.values.map(topic => TopicKey(topic.name)).toBuffer
// try to complete the request immediately, otherwise put it into the purgatory // try to complete the request immediately, otherwise put it into the purgatory
topicPurgatory.tryCompleteElseWatch(delayedCreate, delayedCreateKeys) topicPurgatory.tryCompleteElseWatch(delayedCreate, delayedCreateKeys)
} }
@ -224,20 +237,24 @@ class AdminManager(val config: KafkaConfig,
*/ */
def deleteTopics(timeout: Int, def deleteTopics(timeout: Int,
topics: Set[String], topics: Set[String],
controllerMutationQuota: ControllerMutationQuota,
responseCallback: Map[String, Errors] => Unit): Unit = { responseCallback: Map[String, Errors] => Unit): Unit = {
// 1. map over topics calling the asynchronous delete // 1. map over topics calling the asynchronous delete
val metadata = topics.map { topic => val metadata = topics.map { topic =>
try { try {
controllerMutationQuota.record(metadataCache.numPartitions(topic).getOrElse(0).toDouble)
adminZkClient.deleteTopic(topic) adminZkClient.deleteTopic(topic)
DeleteTopicMetadata(topic, Errors.NONE) DeleteTopicMetadata(topic, Errors.NONE)
} catch { } catch {
case _: TopicAlreadyMarkedForDeletionException => case _: TopicAlreadyMarkedForDeletionException =>
// swallow the exception, and still track deletion allowing multiple calls to wait for deletion // swallow the exception, and still track deletion allowing multiple calls to wait for deletion
DeleteTopicMetadata(topic, Errors.NONE) 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 => case e: Throwable =>
error(s"Error processing delete topic request for topic $topic", e) 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 { } else {
// 3. else pass the topics and errors to the delayed operation and set the keys // 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 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 // try to complete the request immediately, otherwise put it into the purgatory
topicPurgatory.tryCompleteElseWatch(delayedDelete, delayedDeleteKeys) topicPurgatory.tryCompleteElseWatch(delayedDelete, delayedDeleteKeys)
} }
@ -264,15 +281,15 @@ class AdminManager(val config: KafkaConfig,
def createPartitions(timeout: Int, def createPartitions(timeout: Int,
newPartitions: Seq[CreatePartitionsTopic], newPartitions: Seq[CreatePartitionsTopic],
validateOnly: Boolean, validateOnly: Boolean,
listenerName: ListenerName, controllerMutationQuota: ControllerMutationQuota,
callback: Map[String, ApiError] => Unit): Unit = { callback: Map[String, ApiError] => Unit): Unit = {
val allBrokers = adminZkClient.getBrokerMetadatas() val allBrokers = adminZkClient.getBrokerMetadatas()
val allBrokerIds = allBrokers.map(_.id) val allBrokerIds = allBrokers.map(_.id)
// 1. map over topics creating assignment and calling AdminUtils // 1. map over topics creating assignment and calling AdminUtils
val metadata = newPartitions.map { newPartition => val metadata = newPartitions.map { newPartition =>
val topic = newPartition.name val topic = newPartition.name
try { try {
val existingAssignment = zkClient.getFullReplicaAssignmentForTopics(immutable.Set(topic)).map { val existingAssignment = zkClient.getFullReplicaAssignmentForTopics(immutable.Set(topic)).map {
case (topicPartition, assignment) => case (topicPartition, assignment) =>
@ -296,34 +313,44 @@ class AdminManager(val config: KafkaConfig,
throw new InvalidPartitionsException(s"Topic already has $oldNumPartitions partitions.") throw new InvalidPartitionsException(s"Topic already has $oldNumPartitions partitions.")
} }
val newPartitionsAssignment = Option(newPartition.assignments) val newPartitionsAssignment = Option(newPartition.assignments).map { assignmentMap =>
.map { assignmentMap => val assignments = assignmentMap.asScala.map {
val assignments = assignmentMap.asScala.map { createPartitionAssignment => createPartitionAssignment.brokerIds.asScala.map(_.toInt)
createPartitionAssignment => createPartitionAssignment.brokerIds.asScala.map(_.toInt) }
} val unknownBrokers = assignments.flatten.toSet -- allBrokerIds
val unknownBrokers = assignments.flatten.toSet -- allBrokerIds if (unknownBrokers.nonEmpty)
if (unknownBrokers.nonEmpty) throw new InvalidReplicaAssignmentException(
throw new InvalidReplicaAssignmentException( s"Unknown broker(s) in replica assignment: ${unknownBrokers.mkString(", ")}.")
s"Unknown broker(s) in replica assignment: ${unknownBrokers.mkString(", ")}.")
if (assignments.size != numPartitionsIncrement) if (assignments.size != numPartitionsIncrement)
throw new InvalidReplicaAssignmentException( throw new InvalidReplicaAssignmentException(
s"Increasing the number of partitions by $numPartitionsIncrement " + s"Increasing the number of partitions by $numPartitionsIncrement " +
s"but ${assignments.size} assignments provided.") s"but ${assignments.size} assignments provided.")
assignments.zipWithIndex.map { case (replicas, index) => assignments.zipWithIndex.map { case (replicas, index) =>
existingAssignment.size + index -> replicas existingAssignment.size + index -> replicas
}.toMap }.toMap
} }
val updatedReplicaAssignment = adminZkClient.addPartitions(topic, existingAssignment, allBrokers, val assignmentForNewPartitions = adminZkClient.createNewPartitionsAssignment(
newPartition.count, newPartitionsAssignment, validateOnly = validateOnly) topic, existingAssignment, allBrokers, newPartition.count, newPartitionsAssignment)
CreatePartitionsMetadata(topic, updatedReplicaAssignment.keySet, ApiError.NONE)
if (validateOnly) {
CreatePartitionsMetadata(topic, (existingAssignment ++ assignmentForNewPartitions).keySet)
} else {
controllerMutationQuota.record(numPartitionsIncrement)
val updatedReplicaAssignment = adminZkClient.createPartitionsWithAssignment(
topic, existingAssignment, assignmentForNewPartitions)
CreatePartitionsMetadata(topic, updatedReplicaAssignment.keySet)
}
} catch { } catch {
case e: AdminOperationException => 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 => 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 => resourceToConfigNames.map { case resource =>
def allConfigs(config: AbstractConfig) = { def allConfigs(config: AbstractConfig) = {

View File

@ -44,30 +44,24 @@ case class ClientSensors(metricTags: Map[String, String], quotaSensor: Sensor, t
/** /**
* Configuration settings for quota management * 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 * dynamic defaults or user quotas are not set
* @param numQuotaSamples The number of samples to retain in memory * @param numQuotaSamples The number of samples to retain in memory
* @param quotaWindowSizeSeconds The time span of each sample * @param quotaWindowSizeSeconds The time span of each sample
* *
*/ */
case class ClientQuotaManagerConfig(quotaBytesPerSecondDefault: Long = case class ClientQuotaManagerConfig(quotaDefault: Long =
ClientQuotaManagerConfig.QuotaBytesPerSecondDefault, ClientQuotaManagerConfig.QuotaDefault,
numQuotaSamples: Int = numQuotaSamples: Int =
ClientQuotaManagerConfig.DefaultNumQuotaSamples, ClientQuotaManagerConfig.DefaultNumQuotaSamples,
quotaWindowSizeSeconds: Int = quotaWindowSizeSeconds: Int =
ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds) ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds)
object ClientQuotaManagerConfig { object ClientQuotaManagerConfig {
val QuotaBytesPerSecondDefault = Long.MaxValue val QuotaDefault = Long.MaxValue
// Always have 10 whole windows + 1 current window // Always have 10 whole windows + 1 current window
val DefaultNumQuotaSamples = 11 val DefaultNumQuotaSamples = 11
val DefaultQuotaWindowSizeSeconds = 1 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 { object QuotaTypes {
@ -79,6 +73,9 @@ object QuotaTypes {
} }
object ClientQuotaManager { object ClientQuotaManager {
// Purge sensors after 1 hour of inactivity
val InactiveSensorExpirationTimeSeconds = 3600
val DefaultClientIdQuotaEntity = KafkaQuotaEntity(None, Some(DefaultClientIdEntity)) val DefaultClientIdQuotaEntity = KafkaQuotaEntity(None, Some(DefaultClientIdEntity))
val DefaultUserQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), None) val DefaultUserQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), None)
val DefaultUserClientIdQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), Some(DefaultClientIdEntity)) val DefaultUserClientIdQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), Some(DefaultClientIdEntity))
@ -111,10 +108,12 @@ object ClientQuotaManager {
clientIdEntity: Option[ClientQuotaEntity.ConfigEntity]) extends ClientQuotaEntity { clientIdEntity: Option[ClientQuotaEntity.ConfigEntity]) extends ClientQuotaEntity {
override def configEntities: util.List[ClientQuotaEntity.ConfigEntity] = override def configEntities: util.List[ClientQuotaEntity.ConfigEntity] =
(userEntity.toList ++ clientIdEntity.toList).asJava (userEntity.toList ++ clientIdEntity.toList).asJava
def sanitizedUser: String = userEntity.map { def sanitizedUser: String = userEntity.map {
case entity: UserEntity => entity.sanitizedUser case entity: UserEntity => entity.sanitizedUser
case DefaultUserEntity => ConfigEntityName.Default case DefaultUserEntity => ConfigEntityName.Default
}.getOrElse("") }.getOrElse("")
def clientId: String = clientIdEntity.map(_.name).getOrElse("") def clientId: String = clientIdEntity.map(_.name).getOrElse("")
override def toString: String = { override def toString: String = {
@ -128,6 +127,32 @@ object ClientQuotaManager {
val User = "user" val User = "user"
val ClientId = "client-id" 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 metrics @Metrics Metrics instance
* @param quotaType Quota type of this quota manager * @param quotaType Quota type of this quota manager
* @param time @Time object to use * @param time @Time object to use
* @param threadNamePrefix The thread prefix to use
* @param clientQuotaCallback An optional @ClientQuotaCallback
*/ */
class ClientQuotaManager(private val config: ClientQuotaManagerConfig, class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
private val metrics: Metrics, private val metrics: Metrics,
private val quotaType: QuotaType, private val quotaType: QuotaType,
private val time: Time, private val time: Time,
threadNamePrefix: String, private val threadNamePrefix: String,
clientQuotaCallback: Option[ClientQuotaCallback] = None) extends Logging { private val clientQuotaCallback: Option[ClientQuotaCallback] = None) extends Logging {
private val staticConfigClientIdQuota = Quota.upperBound(config.quotaBytesPerSecondDefault.toDouble)
private val clientQuotaType = quotaTypeToClientQuotaType(quotaType) private val lock = new ReentrantReadWriteLock()
@volatile private var quotaTypesEnabled = clientQuotaCallback match { 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 Some(_) => QuotaTypes.CustomQuotas
case None => case None =>
if (config.quotaBytesPerSecondDefault == Long.MaxValue) QuotaTypes.NoQuotas if (config.quotaDefault == Long.MaxValue) QuotaTypes.NoQuotas
else QuotaTypes.ClientIdQuotaEnabled 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") private val delayQueueSensor = metrics.sensor(quotaType.toString + "-delayQueue")
delayQueueSensor.add(metrics.metricName("queue-size", delayQueueSensor.add(metrics.metricName("queue-size", quotaType.toString,
quotaType.toString,
"Tracks the size of the delay queue"), new CumulativeSum()) "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 start() // Use start method to keep spotbugs happy
private def start(): Unit = { private def start(): Unit = {
throttledChannelReaper.start() throttledChannelReaper.start()
@ -213,17 +243,15 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
def quotasEnabled: Boolean = quotaTypesEnabled != QuotaTypes.NoQuotas def quotasEnabled: Boolean = quotaTypesEnabled != QuotaTypes.NoQuotas
/** /**
* Records that a user/clientId changed produced/consumed bytes being throttled at the specified time. If quota has * See {recordAndGetThrottleTimeMs}.
* 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
*/
def maybeRecordAndGetThrottleTimeMs(request: RequestChannel.Request, value: Double, timeMs: Long): Int = { def maybeRecordAndGetThrottleTimeMs(request: RequestChannel.Request, value: Double, timeMs: Long): Int = {
maybeRecordAndGetThrottleTimeMs(request.session, request.header.clientId, value, timeMs) maybeRecordAndGetThrottleTimeMs(request.session, request.header.clientId, value, timeMs)
} }
/**
* See {recordAndGetThrottleTimeMs}.
*/
def maybeRecordAndGetThrottleTimeMs(session: Session, clientId: String, value: Double, timeMs: Long): Int = { def maybeRecordAndGetThrottleTimeMs(session: Session, clientId: String, value: Double, timeMs: Long): Int = {
// Record metrics only if quotas are enabled. // Record metrics only if quotas are enabled.
if (quotasEnabled) { 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. * 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 * 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. * 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 request client request
* @param channelThrottlingCallback Callback for channel throttling * @param throttleTimeMs Duration in milliseconds for which the channel is to be muted.
* @return ThrottledChannel object * @param channelThrottlingCallback Callback for channel throttling
*/ */
def throttle(request: RequestChannel.Request, throttleTimeMs: Int, channelThrottlingCallback: Response => Unit): Unit = { def throttle(request: RequestChannel.Request, throttleTimeMs: Int, channelThrottlingCallback: Response => Unit): Unit = {
if (throttleTimeMs > 0) { if (throttleTimeMs > 0) {
val clientSensors = getOrCreateQuotaSensors(request.session, request.header.clientId) 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. * 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) Option(quotaCallback.quotaLimit(clientQuotaType, metricTags)).map(_.toDouble).getOrElse(Long.MaxValue)
} }
/* /**
* This calculates the amount of time needed to bring the metric within quota * This calculates the amount of time needed to bring the metric within quota
* assuming that no new metrics are recorded. * 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, * See {ClientQuotaManager.throttleTime} for the details.
* 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.
*/ */
protected def throttleTime(quotaValue: Double, quotaBound: Double, windowSize: Long): Long = { protected def throttleTime(e: QuotaViolationException, timeMs: Long): Long = {
val difference = quotaValue - quotaBound ClientQuotaManager.throttleTime(e, timeMs)
// Use the precise window used by the rate calculation
val throttleTimeMs = difference / quotaBound * windowSize
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")
}
}
/*
* This function either returns the sensors for a given client id or creates them if they don't exist * 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 * 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, metricTags,
sensorAccessor.getOrCreate( sensorAccessor.getOrCreate(
getQuotaSensorName(metricTags), getQuotaSensorName(metricTags),
ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds, ClientQuotaManager.InactiveSensorExpirationTimeSeconds,
clientRateMetricName(metricTags), clientRateMetricName(metricTags),
Some(getQuotaMetricConfig(metricTags)), Some(getQuotaMetricConfig(metricTags)),
new Rate new Rate
), ),
sensorAccessor.getOrCreate(getThrottleTimeSensorName(metricTags), sensorAccessor.getOrCreate(
ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds, getThrottleTimeSensorName(metricTags),
ClientQuotaManager.InactiveSensorExpirationTimeSeconds,
throttleMetricName(metricTags), throttleMetricName(metricTags),
None, None,
new Avg new Avg
@ -408,7 +434,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
protected def getOrCreateSensor(sensorName: String, metricName: MetricName): Sensor = { protected def getOrCreateSensor(sensorName: String, metricName: MetricName): Sensor = {
sensorAccessor.getOrCreate( sensorAccessor.getOrCreate(
sensorName, sensorName,
ClientQuotaManagerConfig.InactiveSensorExpirationTimeSeconds, ClientQuotaManager.InactiveSensorExpirationTimeSeconds,
metricName, metricName,
None, None,
new Rate 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 * 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 * or when partitions leaders change and custom callbacks that implement partition-based quotas
* have updated quotas. * have updated quotas.
*
* @param updatedQuotaEntity If set to one entity and quotas have only been enabled at one * @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, * 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 * 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) 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 = { def shutdown(): Unit = {
throttledChannelReaper.shutdown() throttledChannelReaper.shutdown()
} }

View File

@ -26,15 +26,25 @@ import org.apache.kafka.server.quota.ClientQuotaCallback
import scala.jdk.CollectionConverters._ 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, class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig,
private val metrics: Metrics, private val metrics: Metrics,
private val time: Time, private val time: Time,
threadNamePrefix: String, private val threadNamePrefix: String,
quotaCallback: Option[ClientQuotaCallback]) private val quotaCallback: Option[ClientQuotaCallback])
extends ClientQuotaManager(config, metrics, QuotaType.Request, time, threadNamePrefix, quotaCallback) { extends ClientQuotaManager(config, metrics, QuotaType.Request, time, threadNamePrefix, quotaCallback) {
val maxThrottleTimeMs = TimeUnit.SECONDS.toMillis(this.config.quotaWindowSizeSeconds)
def exemptSensor = getOrCreateSensor(exemptSensorName, exemptMetricName) 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 = { def recordExempt(value: Double): Unit = {
exemptSensor.record(value) exemptSensor.record(value)
@ -49,7 +59,7 @@ class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig,
def maybeRecordAndGetThrottleTimeMs(request: RequestChannel.Request, timeMs: Long): Int = { def maybeRecordAndGetThrottleTimeMs(request: RequestChannel.Request, timeMs: Long): Int = {
if (quotasEnabled) { if (quotasEnabled) {
request.recordNetworkThreadTimeCallback = Some(timeNanos => recordNoThrottle( 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, recordAndGetThrottleTimeMs(request.session, request.header.clientId,
nanosToPercentage(request.requestThreadTimeNanos), timeMs) nanosToPercentage(request.requestThreadTimeNanos), timeMs)
} else { } else {
@ -64,8 +74,8 @@ class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig,
} }
} }
override protected def throttleTime(quotaValue: Double, quotaBound: Double, windowSize: Long): Long = { override protected def throttleTime(e: QuotaViolationException, timeMs: Long): Long = {
math.min(super.throttleTime(quotaValue, quotaBound, windowSize), maxThrottleTimeMs) math.min(super.throttleTime(e, timeMs), maxThrottleTimeMs)
} }
override protected def clientRateMetricName(quotaMetricTags: Map[String, String]): MetricName = { override protected def clientRateMetricName(quotaMetricTags: Map[String, String]): MetricName = {
@ -74,13 +84,6 @@ class ClientRequestQuotaManager(private val config: ClientQuotaManagerConfig,
quotaMetricTags.asJava) quotaMetricTags.asJava)
} }
private def exemptMetricName: MetricName = { private def nanosToPercentage(nanos: Long): Double =
metrics.metricName("exempt-request-time", QuotaType.Request.toString, nanos * ClientRequestQuotaManager.NanosToPercentagePerSecond
"Tracking exempt-request-time utilization percentage")
}
private def exemptSensorName: String = "exempt-" + QuotaType.Request
private def nanosToPercentage(nanos: Long): Double = nanos * ClientQuotaManagerConfig.NanosToPercentagePerSecond
} }

View File

@ -144,6 +144,12 @@ class QuotaConfigHandler(private val quotaManagers: QuotaManagers) {
else else
None None
quotaManagers.request.updateQuota(sanitizedUser, clientId, sanitizedClientId, requestQuota) 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)
} }
} }

View File

@ -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)
}
}

View File

@ -28,6 +28,20 @@ import scala.collection._
*/ */
case class CreatePartitionsMetadata(topic: String, partitions: Set[Int], error: ApiError) 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. * A delayed create topic or create partitions operation that is stored in the topic purgatory.
*/ */

View File

@ -26,6 +26,12 @@ import scala.collection._
*/ */
case class DeleteTopicMetadata(topic: String, error: Errors) 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 * A delayed delete topics operation that can be created by the admin manager and watched
* in the topic purgatory * in the topic purgatory

View File

@ -35,15 +35,15 @@ import scala.jdk.CollectionConverters._
object DynamicConfig { object DynamicConfig {
object Broker { object Broker {
//Properties // Properties
val LeaderReplicationThrottledRateProp = "leader.replication.throttled.rate" val LeaderReplicationThrottledRateProp = "leader.replication.throttled.rate"
val FollowerReplicationThrottledRateProp = "follower.replication.throttled.rate" val FollowerReplicationThrottledRateProp = "follower.replication.throttled.rate"
val ReplicaAlterLogDirsIoMaxBytesPerSecondProp = "replica.alter.log.dirs.io.max.bytes.per.second" val ReplicaAlterLogDirsIoMaxBytesPerSecondProp = "replica.alter.log.dirs.io.max.bytes.per.second"
//Defaults // Defaults
val DefaultReplicationThrottledRate = ReplicationQuotaManagerConfig.QuotaBytesPerSecondDefault val DefaultReplicationThrottledRate = ReplicationQuotaManagerConfig.QuotaBytesPerSecondDefault
//Documentation // Documentation
val LeaderReplicationThrottledRateDoc = "A long representing the upper bound (bytes/sec) on replication traffic for leaders enumerated in the " + 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"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." 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. " + 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." 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() 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(LeaderReplicationThrottledRateProp, LONG, DefaultReplicationThrottledRate, atLeast(0), MEDIUM, LeaderReplicationThrottledRateDoc)
.define(FollowerReplicationThrottledRateProp, LONG, DefaultReplicationThrottledRate, atLeast(0), MEDIUM, FollowerReplicationThrottledRateDoc) .define(FollowerReplicationThrottledRateProp, LONG, DefaultReplicationThrottledRate, atLeast(0), MEDIUM, FollowerReplicationThrottledRateDoc)
.define(ReplicaAlterLogDirsIoMaxBytesPerSecondProp, LONG, DefaultReplicationThrottledRate, atLeast(0), MEDIUM, ReplicaAlterLogDirsIoMaxBytesPerSecondDoc) .define(ReplicaAlterLogDirsIoMaxBytesPerSecondProp, LONG, DefaultReplicationThrottledRate, atLeast(0), MEDIUM, ReplicaAlterLogDirsIoMaxBytesPerSecondDoc)
@ -71,32 +71,39 @@ object DynamicConfig {
val ProducerByteRateOverrideProp = "producer_byte_rate" val ProducerByteRateOverrideProp = "producer_byte_rate"
val ConsumerByteRateOverrideProp = "consumer_byte_rate" val ConsumerByteRateOverrideProp = "consumer_byte_rate"
val RequestPercentageOverrideProp = "request_percentage" 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) def isQuotaConfig(name: String): Boolean = configNames.contains(name)
} }
object Client { object Client {
//Properties // Properties
val ProducerByteRateOverrideProp = QuotaConfigs.ProducerByteRateOverrideProp val ProducerByteRateOverrideProp = QuotaConfigs.ProducerByteRateOverrideProp
val ConsumerByteRateOverrideProp = QuotaConfigs.ConsumerByteRateOverrideProp val ConsumerByteRateOverrideProp = QuotaConfigs.ConsumerByteRateOverrideProp
val RequestPercentageOverrideProp = QuotaConfigs.RequestPercentageOverrideProp val RequestPercentageOverrideProp = QuotaConfigs.RequestPercentageOverrideProp
val ControllerMutationOverrideProp = QuotaConfigs.ControllerMutationOverrideProp
//Defaults // Defaults
val DefaultProducerOverride = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault val DefaultProducerOverride = ClientQuotaManagerConfig.QuotaDefault
val DefaultConsumerOverride = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault val DefaultConsumerOverride = ClientQuotaManagerConfig.QuotaDefault
val DefaultRequestOverride = ClientQuotaManagerConfig.QuotaRequestPercentDefault val DefaultRequestOverride = ClientRequestQuotaManager.QuotaRequestPercentDefault
val DefaultControllerMutationOverride = ClientQuotaManagerConfig.QuotaDefault
//Documentation // Documentation
val ProducerOverrideDoc = "A rate representing the upper bound (bytes/sec) for producer traffic." 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 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 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() private val clientConfigs = new ConfigDef()
.define(ProducerByteRateOverrideProp, LONG, DefaultProducerOverride, MEDIUM, ProducerOverrideDoc) .define(ProducerByteRateOverrideProp, LONG, DefaultProducerOverride, MEDIUM, ProducerOverrideDoc)
.define(ConsumerByteRateOverrideProp, LONG, DefaultConsumerOverride, MEDIUM, ConsumerOverrideDoc) .define(ConsumerByteRateOverrideProp, LONG, DefaultConsumerOverride, MEDIUM, ConsumerOverrideDoc)
.define(RequestPercentageOverrideProp, DOUBLE, DefaultRequestOverride, MEDIUM, RequestOverrideDoc) .define(RequestPercentageOverrideProp, DOUBLE, DefaultRequestOverride, MEDIUM, RequestOverrideDoc)
.define(ControllerMutationOverrideProp, LONG, DefaultConsumerOverride, MEDIUM, ControllerMutationOverrideDoc)
def configKeys = clientConfigs.configKeys def configKeys = clientConfigs.configKeys
@ -106,12 +113,12 @@ object DynamicConfig {
} }
object User { object User {
// Definitions
//Definitions
private val userConfigs = CredentialProvider.userCredentialConfigs private val userConfigs = CredentialProvider.userCredentialConfigs
.define(Client.ProducerByteRateOverrideProp, LONG, Client.DefaultProducerOverride, MEDIUM, Client.ProducerOverrideDoc) .define(Client.ProducerByteRateOverrideProp, LONG, Client.DefaultProducerOverride, MEDIUM, Client.ProducerOverrideDoc)
.define(Client.ConsumerByteRateOverrideProp, LONG, Client.DefaultConsumerOverride, MEDIUM, Client.ConsumerOverrideDoc) .define(Client.ConsumerByteRateOverrideProp, LONG, Client.DefaultConsumerOverride, MEDIUM, Client.ConsumerOverrideDoc)
.define(Client.RequestPercentageOverrideProp, DOUBLE, Client.DefaultRequestOverride, MEDIUM, Client.RequestOverrideDoc) .define(Client.RequestPercentageOverrideProp, DOUBLE, Client.DefaultRequestOverride, MEDIUM, Client.RequestOverrideDoc)
.define(Client.ControllerMutationOverrideProp, LONG, Client.DefaultConsumerOverride, MEDIUM, Client.ControllerMutationOverrideDoc)
def configKeys = userConfigs.configKeys def configKeys = userConfigs.configKeys
@ -121,7 +128,7 @@ object DynamicConfig {
} }
private def validate(configDef: ConfigDef, props: Properties, customPropsAllowed: Boolean) = { private def validate(configDef: ConfigDef, props: Properties, customPropsAllowed: Boolean) = {
//Validate Names // Validate Names
val names = configDef.names() val names = configDef.names()
val propKeys = props.keySet.asScala.map(_.asInstanceOf[String]) val propKeys = props.keySet.asScala.map(_.asInstanceOf[String])
if (!customPropsAllowed) { if (!customPropsAllowed) {
@ -129,7 +136,7 @@ object DynamicConfig {
require(unknownKeys.isEmpty, s"Unknown Dynamic Configuration: $unknownKeys.") require(unknownKeys.isEmpty, s"Unknown Dynamic Configuration: $unknownKeys.")
} }
val propResolved = DynamicBrokerConfig.resolveVariableConfigs(props) val propResolved = DynamicBrokerConfig.resolveVariableConfigs(props)
//ValidateValues // ValidateValues
configDef.parse(propResolved) configDef.parse(propResolved)
} }
} }

View File

@ -312,6 +312,7 @@ class KafkaApis(val requestChannel: RequestChannel,
quotas.fetch.updateQuotaMetricConfigs() quotas.fetch.updateQuotaMetricConfigs()
quotas.produce.updateQuotaMetricConfigs() quotas.produce.updateQuotaMetricConfigs()
quotas.request.updateQuotaMetricConfigs() quotas.request.updateQuotaMetricConfigs()
quotas.controllerMutation.updateQuotaMetricConfigs()
} }
} }
if (replicaManager.hasDelayedElectionOperations) { if (replicaManager.hasDelayedElectionOperations) {
@ -1684,25 +1685,27 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
def handleCreateTopicsRequest(request: RequestChannel.Request): Unit = { def handleCreateTopicsRequest(request: RequestChannel.Request): Unit = {
val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 6)
def sendResponseCallback(results: CreatableTopicResultCollection): Unit = { def sendResponseCallback(results: CreatableTopicResultCollection): Unit = {
def createResponse(requestThrottleMs: Int): AbstractResponse = { def createResponse(requestThrottleMs: Int): AbstractResponse = {
val responseData = new CreateTopicsResponseData(). val responseData = new CreateTopicsResponseData()
setThrottleTimeMs(requestThrottleMs). .setThrottleTimeMs(requestThrottleMs)
setTopics(results) .setTopics(results)
val responseBody = new CreateTopicsResponse(responseData) val responseBody = new CreateTopicsResponse(responseData)
trace(s"Sending create topics response $responseData for correlation id " + trace(s"Sending create topics response $responseData for correlation id " +
s"${request.header.correlationId} to client ${request.header.clientId}.") s"${request.header.correlationId} to client ${request.header.clientId}.")
responseBody responseBody
} }
sendResponseMaybeThrottle(request, createResponse) sendResponseMaybeThrottle(controllerMutationQuota, request, createResponse, onComplete = None)
} }
val createTopicsRequest = request.body[CreateTopicsRequest] val createTopicsRequest = request.body[CreateTopicsRequest]
val results = new CreatableTopicResultCollection(createTopicsRequest.data.topics.size) val results = new CreatableTopicResultCollection(createTopicsRequest.data.topics.size)
if (!controller.isActive) { if (!controller.isActive) {
createTopicsRequest.data.topics.forEach { topic => createTopicsRequest.data.topics.forEach { topic =>
results.add(new CreatableTopicResult().setName(topic.name). results.add(new CreatableTopicResult().setName(topic.name)
setErrorCode(Errors.NOT_CONTROLLER.code)) .setErrorCode(Errors.NOT_CONTROLLER.code))
} }
sendResponseCallback(results) sendResponseCallback(results)
} else { } else {
@ -1718,7 +1721,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val authorizedForDescribeConfigs = filterByAuthorized(request.context, DESCRIBE_CONFIGS, TOPIC, val authorizedForDescribeConfigs = filterByAuthorized(request.context, DESCRIBE_CONFIGS, TOPIC,
topics, logIfDenied = false)(identity).map(name => name -> results.find(name)).toMap topics, logIfDenied = false)(identity).map(name => name -> results.find(name)).toMap
results.forEach(topic => { results.forEach { topic =>
if (results.findAll(topic.name).size > 1) { if (results.findAll(topic.name).size > 1) {
topic.setErrorCode(Errors.INVALID_REQUEST.code) topic.setErrorCode(Errors.INVALID_REQUEST.code)
topic.setErrorMessage("Found multiple entries for this topic.") topic.setErrorMessage("Found multiple entries for this topic.")
@ -1729,7 +1732,7 @@ class KafkaApis(val requestChannel: RequestChannel,
if (!authorizedForDescribeConfigs.contains(topic.name)) { if (!authorizedForDescribeConfigs.contains(topic.name)) {
topic.setTopicConfigErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) topic.setTopicConfigErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code)
} }
}) }
val toCreate = mutable.Map[String, CreatableTopic]() val toCreate = mutable.Map[String, CreatableTopic]()
createTopicsRequest.data.topics.forEach { topic => createTopicsRequest.data.topics.forEach { topic =>
if (results.find(topic.name).errorCode == Errors.NONE.code) { if (results.find(topic.name).errorCode == Errors.NONE.code) {
@ -1746,21 +1749,24 @@ class KafkaApis(val requestChannel: RequestChannel,
result.setConfigs(List.empty.asJava) result.setConfigs(List.empty.asJava)
.setNumPartitions(-1) .setNumPartitions(-1)
.setReplicationFactor(-1) .setReplicationFactor(-1)
.setTopicConfigErrorCode(0.toShort) .setTopicConfigErrorCode(Errors.NONE.code)
} }
} }
sendResponseCallback(results) sendResponseCallback(results)
} }
adminManager.createTopics(createTopicsRequest.data.timeoutMs, adminManager.createTopics(
createTopicsRequest.data.validateOnly, createTopicsRequest.data.timeoutMs,
toCreate, createTopicsRequest.data.validateOnly,
authorizedForDescribeConfigs, toCreate,
handleCreateTopicsResults) authorizedForDescribeConfigs,
controllerMutationQuota,
handleCreateTopicsResults)
} }
} }
def handleCreatePartitionsRequest(request: RequestChannel.Request): Unit = { def handleCreatePartitionsRequest(request: RequestChannel.Request): Unit = {
val createPartitionsRequest = request.body[CreatePartitionsRequest] val createPartitionsRequest = request.body[CreatePartitionsRequest]
val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 3)
def sendResponseCallback(results: Map[String, ApiError]): Unit = { def sendResponseCallback(results: Map[String, ApiError]): Unit = {
def createResponse(requestThrottleMs: Int): AbstractResponse = { def createResponse(requestThrottleMs: Int): AbstractResponse = {
@ -1777,7 +1783,7 @@ class KafkaApis(val requestChannel: RequestChannel,
s"client ${request.header.clientId}.") s"client ${request.header.clientId}.")
responseBody responseBody
} }
sendResponseMaybeThrottle(request, createResponse) sendResponseMaybeThrottle(controllerMutationQuota, request, createResponse, onComplete = None)
} }
if (!controller.isActive) { 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.")) ++ 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.")) 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, valid,
createPartitionsRequest.data.validateOnly, createPartitionsRequest.data.validateOnly,
request.context.listenerName, result => sendResponseCallback(result ++ errors)) controllerMutationQuota,
result => sendResponseCallback(result ++ errors))
} }
} }
def handleDeleteTopicsRequest(request: RequestChannel.Request): Unit = { def handleDeleteTopicsRequest(request: RequestChannel.Request): Unit = {
val controllerMutationQuota = quotas.controllerMutation.newQuotaFor(request, strictSinceVersion = 5)
def sendResponseCallback(results: DeletableTopicResultCollection): Unit = { def sendResponseCallback(results: DeletableTopicResultCollection): Unit = {
def createResponse(requestThrottleMs: Int): AbstractResponse = { def createResponse(requestThrottleMs: Int): AbstractResponse = {
val responseData = new DeleteTopicsResponseData() 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}.") trace(s"Sending delete topics response $responseBody for correlation id ${request.header.correlationId} to client ${request.header.clientId}.")
responseBody responseBody
} }
sendResponseMaybeThrottle(request, createResponse) sendResponseMaybeThrottle(controllerMutationQuota, request, createResponse, onComplete = None)
} }
val deleteTopicRequest = request.body[DeleteTopicsRequest] val deleteTopicRequest = request.body[DeleteTopicsRequest]
@ -1872,6 +1882,7 @@ class KafkaApis(val requestChannel: RequestChannel,
adminManager.deleteTopics( adminManager.deleteTopics(
deleteTopicRequest.data.timeoutMs, deleteTopicRequest.data.timeoutMs,
toDelete, toDelete,
controllerMutationQuota,
handleDeleteTopicsResults handleDeleteTopicsResults
) )
} }
@ -3090,6 +3101,30 @@ class KafkaApis(val requestChannel: RequestChannel,
throttleTimeMs 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, private def sendResponseExemptThrottle(request: RequestChannel.Request,
response: AbstractResponse, response: AbstractResponse,
onComplete: Option[Send => Unit] = None): Unit = { onComplete: Option[Send => Unit] = None): Unit = {

View File

@ -196,14 +196,16 @@ object Defaults {
val FetchMaxBytes = 55 * 1024 * 1024 val FetchMaxBytes = 55 * 1024 * 1024
/** ********* Quota Configuration ***********/ /** ********* Quota Configuration ***********/
val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaDefault
val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaDefault
val NumQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples val NumQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples
val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds
val NumReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DefaultNumQuotaSamples val NumReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DefaultNumQuotaSamples
val ReplicationQuotaWindowSizeSeconds: Int = ReplicationQuotaManagerConfig.DefaultQuotaWindowSizeSeconds val ReplicationQuotaWindowSizeSeconds: Int = ReplicationQuotaManagerConfig.DefaultQuotaWindowSizeSeconds
val NumAlterLogDirsReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DefaultNumQuotaSamples val NumAlterLogDirsReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DefaultNumQuotaSamples
val AlterLogDirsReplicationQuotaWindowSizeSeconds: Int = ReplicationQuotaManagerConfig.DefaultQuotaWindowSizeSeconds val AlterLogDirsReplicationQuotaWindowSizeSeconds: Int = ReplicationQuotaManagerConfig.DefaultQuotaWindowSizeSeconds
val NumControllerQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples
val ControllerQuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds
/** ********* Transaction Configuration ***********/ /** ********* Transaction Configuration ***********/
val TransactionalIdExpirationMsDefault = 604800000 val TransactionalIdExpirationMsDefault = 604800000
@ -483,9 +485,11 @@ object KafkaConfig {
val NumQuotaSamplesProp = "quota.window.num" val NumQuotaSamplesProp = "quota.window.num"
val NumReplicationQuotaSamplesProp = "replication.quota.window.num" val NumReplicationQuotaSamplesProp = "replication.quota.window.num"
val NumAlterLogDirsReplicationQuotaSamplesProp = "alter.log.dirs.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 QuotaWindowSizeSecondsProp = "quota.window.size.seconds"
val ReplicationQuotaWindowSizeSecondsProp = "replication.quota.window.size.seconds" val ReplicationQuotaWindowSizeSecondsProp = "replication.quota.window.size.seconds"
val AlterLogDirsReplicationQuotaWindowSizeSecondsProp = "alter.log.dirs.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 ClientQuotaCallbackClassProp = "client.quota.callback.class"
val DeleteTopicEnableProp = "delete.topic.enable" 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 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 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 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 QuotaWindowSizeSecondsDoc = "The time span of each sample for client quotas"
val ReplicationQuotaWindowSizeSecondsDoc = "The time span of each sample for replication 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 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, " + 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> " + "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 " + "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(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc)
.define(NumReplicationQuotaSamplesProp, INT, Defaults.NumReplicationQuotaSamples, atLeast(1), LOW, NumReplicationQuotaSamplesDoc) .define(NumReplicationQuotaSamplesProp, INT, Defaults.NumReplicationQuotaSamples, atLeast(1), LOW, NumReplicationQuotaSamplesDoc)
.define(NumAlterLogDirsReplicationQuotaSamplesProp, INT, Defaults.NumAlterLogDirsReplicationQuotaSamples, atLeast(1), LOW, NumAlterLogDirsReplicationQuotaSamplesDoc) .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(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc)
.define(ReplicationQuotaWindowSizeSecondsProp, INT, Defaults.ReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, ReplicationQuotaWindowSizeSecondsDoc) .define(ReplicationQuotaWindowSizeSecondsProp, INT, Defaults.ReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, ReplicationQuotaWindowSizeSecondsDoc)
.define(AlterLogDirsReplicationQuotaWindowSizeSecondsProp, INT, Defaults.AlterLogDirsReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, AlterLogDirsReplicationQuotaWindowSizeSecondsDoc) .define(AlterLogDirsReplicationQuotaWindowSizeSecondsProp, INT, Defaults.AlterLogDirsReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, AlterLogDirsReplicationQuotaWindowSizeSecondsDoc)
.define(ControllerQuotaWindowSizeSecondsProp, INT, Defaults.ControllerQuotaWindowSizeSeconds, atLeast(1), LOW, ControllerQuotaWindowSizeSecondsDoc)
.define(ClientQuotaCallbackClassProp, CLASS, null, LOW, ClientQuotaCallbackClassDoc) .define(ClientQuotaCallbackClassProp, CLASS, null, LOW, ClientQuotaCallbackClassDoc)
/** ********* General Security Configuration ****************/ /** ********* General Security Configuration ****************/
@ -1600,6 +1609,8 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO
val replicationQuotaWindowSizeSeconds = getInt(KafkaConfig.ReplicationQuotaWindowSizeSecondsProp) val replicationQuotaWindowSizeSeconds = getInt(KafkaConfig.ReplicationQuotaWindowSizeSecondsProp)
val numAlterLogDirsReplicationQuotaSamples = getInt(KafkaConfig.NumAlterLogDirsReplicationQuotaSamplesProp) val numAlterLogDirsReplicationQuotaSamples = getInt(KafkaConfig.NumAlterLogDirsReplicationQuotaSamplesProp)
val alterLogDirsReplicationQuotaWindowSizeSeconds = getInt(KafkaConfig.AlterLogDirsReplicationQuotaWindowSizeSecondsProp) val alterLogDirsReplicationQuotaWindowSizeSeconds = getInt(KafkaConfig.AlterLogDirsReplicationQuotaWindowSizeSecondsProp)
val numControllerQuotaSamples = getInt(KafkaConfig.NumControllerQuotaSamplesProp)
val controllerQuotaWindowSizeSeconds = getInt(KafkaConfig.ControllerQuotaWindowSizeSecondsProp)
/** ********* Fetch Configuration **************/ /** ********* Fetch Configuration **************/
val maxIncrementalFetchSessionCacheSlots = getInt(KafkaConfig.MaxIncrementalFetchSessionCacheSlots) val maxIncrementalFetchSessionCacheSlots = getInt(KafkaConfig.MaxIncrementalFetchSessionCacheSlots)

View File

@ -222,6 +222,10 @@ class MetadataCache(brokerId: Int) extends Logging {
metadataSnapshot.partitionStates.get(topic).flatMap(_.get(partitionId)) 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 not known, return None;
// if the leader is known and corresponding node is available, return Some(node) // 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) // if the leader is known but corresponding node with the listener name is not available, return Some(NO_NODE)

View File

@ -22,15 +22,28 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.server.quota.ClientQuotaCallback import org.apache.kafka.server.quota.ClientQuotaCallback
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.server.quota.ClientQuotaType
object QuotaType { object QuotaType {
case object Fetch extends QuotaType case object Fetch extends QuotaType
case object Produce extends QuotaType case object Produce extends QuotaType
case object Request extends QuotaType case object Request extends QuotaType
case object ControllerMutation extends QuotaType
case object LeaderReplication extends QuotaType case object LeaderReplication extends QuotaType
case object FollowerReplication extends QuotaType case object FollowerReplication extends QuotaType
case object AlterLogDirsReplication 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 sealed trait QuotaType
object QuotaFactory extends Logging { object QuotaFactory extends Logging {
@ -44,6 +57,7 @@ object QuotaFactory extends Logging {
case class QuotaManagers(fetch: ClientQuotaManager, case class QuotaManagers(fetch: ClientQuotaManager,
produce: ClientQuotaManager, produce: ClientQuotaManager,
request: ClientRequestQuotaManager, request: ClientRequestQuotaManager,
controllerMutation: ControllerMutationQuotaManager,
leader: ReplicationQuotaManager, leader: ReplicationQuotaManager,
follower: ReplicationQuotaManager, follower: ReplicationQuotaManager,
alterLogDirs: ReplicationQuotaManager, alterLogDirs: ReplicationQuotaManager,
@ -52,6 +66,7 @@ object QuotaFactory extends Logging {
fetch.shutdown() fetch.shutdown()
produce.shutdown() produce.shutdown()
request.shutdown() request.shutdown()
controllerMutation.shutdown()
clientQuotaCallback.foreach(_.close()) clientQuotaCallback.foreach(_.close())
} }
} }
@ -64,6 +79,8 @@ object QuotaFactory extends Logging {
new ClientQuotaManager(clientFetchConfig(cfg), metrics, Fetch, time, threadNamePrefix, clientQuotaCallback), new ClientQuotaManager(clientFetchConfig(cfg), metrics, Fetch, time, threadNamePrefix, clientQuotaCallback),
new ClientQuotaManager(clientProduceConfig(cfg), metrics, Produce, time, threadNamePrefix, clientQuotaCallback), new ClientQuotaManager(clientProduceConfig(cfg), metrics, Produce, time, threadNamePrefix, clientQuotaCallback),
new ClientRequestQuotaManager(clientRequestConfig(cfg), metrics, 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, LeaderReplication, time),
new ReplicationQuotaManager(replicationConfig(cfg), metrics, FollowerReplication, time), new ReplicationQuotaManager(replicationConfig(cfg), metrics, FollowerReplication, time),
new ReplicationQuotaManager(alterLogDirsReplicationConfig(cfg), metrics, AlterLogDirsReplication, time), new ReplicationQuotaManager(alterLogDirsReplicationConfig(cfg), metrics, AlterLogDirsReplication, time),
@ -75,7 +92,7 @@ object QuotaFactory extends Logging {
if (cfg.producerQuotaBytesPerSecondDefault != Long.MaxValue) 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.") 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( ClientQuotaManagerConfig(
quotaBytesPerSecondDefault = cfg.producerQuotaBytesPerSecondDefault, quotaDefault = cfg.producerQuotaBytesPerSecondDefault,
numQuotaSamples = cfg.numQuotaSamples, numQuotaSamples = cfg.numQuotaSamples,
quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds
) )
@ -85,7 +102,7 @@ object QuotaFactory extends Logging {
if (cfg.consumerQuotaBytesPerSecondDefault != Long.MaxValue) 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.") 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( ClientQuotaManagerConfig(
quotaBytesPerSecondDefault = cfg.consumerQuotaBytesPerSecondDefault, quotaDefault = cfg.consumerQuotaBytesPerSecondDefault,
numQuotaSamples = cfg.numQuotaSamples, numQuotaSamples = cfg.numQuotaSamples,
quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds 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 = { def replicationConfig(cfg: KafkaConfig): ReplicationQuotaManagerConfig = {
ReplicationQuotaManagerConfig( ReplicationQuotaManagerConfig(
numQuotaSamples = cfg.numReplicationQuotaSamples, numQuotaSamples = cfg.numReplicationQuotaSamples,

View File

@ -81,10 +81,21 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
brokerMetadatas.sortBy(_.id) 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, def createTopicWithAssignment(topic: String,
config: Properties, config: Properties,
partitionReplicaAssignment: Map[Int, Seq[Int]]): Unit = { partitionReplicaAssignment: Map[Int, Seq[Int]],
validateTopicCreate(topic, partitionReplicaAssignment, config) validate: Boolean = true): Unit = {
if (validate)
validateTopicCreate(topic, partitionReplicaAssignment, config)
info(s"Creating topic $topic with configuration $config and initial partition " + info(s"Creating topic $topic with configuration $config and initial partition " +
s"assignment $partitionReplicaAssignment") 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, def validateTopicCreate(topic: String,
partitionReplicaAssignment: Map[Int, Seq[Int]], partitionReplicaAssignment: Map[Int, Seq[Int]],
@ -171,22 +187,57 @@ class AdminZkClient(zkClient: KafkaZkClient) extends Logging {
} }
/** /**
* Add partitions to existing topic with optional 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 topic Topic for adding partitions to
* @param allBrokers All brokers in the cluster * @param existingAssignment A map from partition id to its assignment
* @param numPartitions Number of partitions to be set * @param allBrokers All brokers in the cluster
* @param replicaAssignment Manual replica assignment, or none * @param numPartitions Number of partitions to be set
* @param validateOnly If true, validate the parameters without actually adding the partitions * @param replicaAssignment Manual replica assignment, or none
* @return the updated replica assignment * @param validateOnly If true, validate the parameters without actually adding the partitions
*/ * @return the updated replica assignment
*/
def addPartitions(topic: String, def addPartitions(topic: String,
existingAssignment: Map[Int, ReplicaAssignment], existingAssignment: Map[Int, ReplicaAssignment],
allBrokers: Seq[BrokerMetadata], allBrokers: Seq[BrokerMetadata],
numPartitions: Int = 1, numPartitions: Int = 1,
replicaAssignment: Option[Map[Int, Seq[Int]]] = None, replicaAssignment: Option[Map[Int, Seq[Int]]] = None,
validateOnly: Boolean = false): Map[Int, Seq[Int]] = { 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, val existingAssignmentPartition0 = existingAssignment.getOrElse(0,
throw new AdminOperationException( throw new AdminOperationException(
s"Unexpected existing replica assignment for topic '$topic', partition id 0 is missing. " + 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) startIndex, existingAssignment.size)
} }
val proposedAssignment = existingAssignment ++ proposedAssignmentForNewPartitions.map { case (tp, replicas) => proposedAssignmentForNewPartitions.map { case (tp, replicas) =>
tp -> ReplicaAssignment(replicas, List(), List()) 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) /**
} * Add partitions to the existing topic with the provided assignment. This method does
proposedAssignment.map { case (k, v) => k -> v.replicas } * 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]], private def validateReplicaAssignment(replicaAssignment: Map[Int, Seq[Int]],

View File

@ -1601,7 +1601,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
"", "",
"user", "", "user", "",
"client-id", clientId) "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(_, "byte-rate", QuotaType.Produce, producerClientId))
servers.foreach(assertNoMetric(_, "throttle-time", QuotaType.Produce, producerClientId)) servers.foreach(assertNoMetric(_, "throttle-time", QuotaType.Produce, producerClientId))
@ -1615,9 +1615,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
def assertNoExemptRequestMetric(broker: KafkaServer): Unit = { def assertNoExemptRequestMetric(broker: KafkaServer): Unit = {
val metricName = broker.metrics.metricName("exempt-request-time", QuotaType.Request.toString, "") 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 = { def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = {

View File

@ -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)
}
}

View File

@ -17,113 +17,61 @@
package kafka.server package kafka.server
import java.net.InetAddress import java.net.InetAddress
import java.util
import java.util.Collections
import kafka.network.RequestChannel import kafka.network.RequestChannel.Session
import kafka.network.RequestChannel.{EndThrottlingResponse, Session, StartThrottlingResponse}
import kafka.server.QuotaType._ import kafka.server.QuotaType._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.Quota
import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.metrics.{MetricConfig, Metrics, Quota} import org.apache.kafka.common.utils.Sanitizer
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}
class ClientQuotaManagerTest { import org.junit.Assert._
private val time = new MockTime import org.junit.Test
private val metrics = new Metrics(new MetricConfig(), Collections.emptyList(), time)
private val config = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500)
var numCallbacks: Int = 0 class ClientQuotaManagerTest extends BaseClientQuotaManagerTest {
private val config = ClientQuotaManagerConfig(quotaDefault = 500)
@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)
}
private def testQuotaParsing(config: ClientQuotaManagerConfig, client1: UserClient, client2: UserClient, randomClient: UserClient, defaultConfigClient: UserClient): Unit = { 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 { try {
// Case 1: Update the quota. Assert that the new quota value is returned // 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))) clientQuotaManager.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(client2.configUser, client2.configClientId, client2.sanitizedConfigClientId, Some(new Quota(4000, true)))
assertEquals("Default producer quota should be " + config.quotaBytesPerSecondDefault, assertEquals("Default producer quota should be " + config.quotaDefault,
config.quotaBytesPerSecondDefault.toDouble, clientMetrics.quota(randomClient.user, randomClient.clientId).bound, 0.0) config.quotaDefault.toDouble, clientQuotaManager.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 (2000)", 2000, clientQuotaManager.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("Should return the overridden value (4000)", 4000, clientQuotaManager.quota(client2.user, client2.clientId).bound, 0.0)
// p1 should be throttled using the overridden quota // 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) 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. // 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 // p1 should not longer be throttled after the quota change
clientMetrics.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(3000, true))) clientQuotaManager.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) 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) assertEquals(s"throttleTimeMs should be 0. was $throttleTimeMs", 0, throttleTimeMs)
// Case 3: Change quota back to default. Should be throttled again // Case 3: Change quota back to default. Should be throttled again
clientMetrics.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(500, true))) clientQuotaManager.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) 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) 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 // Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled
clientMetrics.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, None) clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, None)
clientMetrics.updateQuota(defaultConfigClient.configUser, defaultConfigClient.configClientId, defaultConfigClient.sanitizedConfigClientId, Some(new Quota(4000, true))) clientQuotaManager.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) 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) assertEquals(s"throttleTimeMs should be 0. was $throttleTimeMs", 0, throttleTimeMs)
} finally { } finally {
clientMetrics.shutdown() clientQuotaManager.shutdown()
} }
} }
@ -150,7 +98,7 @@ class ClientQuotaManagerTest {
val client2 = UserClient("User2", "p2", Some("User2"), None) val client2 = UserClient("User2", "p2", Some("User2"), None)
val randomClient = UserClient("RandomUser", "random-client-id", None, None) val randomClient = UserClient("RandomUser", "random-client-id", None, None)
val defaultConfigClient = UserClient("", "", Some(ConfigEntityName.Default), 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) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient)
} }
@ -164,7 +112,7 @@ class ClientQuotaManagerTest {
val client2 = UserClient("User2", "p2", Some("User2"), Some("p2")) val client2 = UserClient("User2", "p2", Some("User2"), Some("p2"))
val randomClient = UserClient("RandomUser", "random-client-id", None, None) val randomClient = UserClient("RandomUser", "random-client-id", None, None)
val defaultConfigClient = UserClient("", "", Some(ConfigEntityName.Default), Some(ConfigEntityName.Default)) 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) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient)
} }
@ -210,156 +158,155 @@ class ClientQuotaManagerTest {
@Test @Test
def testGetMaxValueInQuotaWindowWithNonDefaultQuotaWindow(): Unit = { def testGetMaxValueInQuotaWindowWithNonDefaultQuotaWindow(): Unit = {
val numFullQuotaWindows = 3 // 3 seconds window (vs. 10 seconds default) val numFullQuotaWindows = 3 // 3 seconds window (vs. 10 seconds default)
val nonDefaultConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = Long.MaxValue, numQuotaSamples = numFullQuotaWindows + 1) val nonDefaultConfig = ClientQuotaManagerConfig(quotaDefault = Long.MaxValue, numQuotaSamples = numFullQuotaWindows + 1)
val quotaManager = new ClientQuotaManager(nonDefaultConfig, metrics, Fetch, time, "") val clientQuotaManager = new ClientQuotaManager(nonDefaultConfig, metrics, Fetch, time, "")
val userSession = Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "userA"), InetAddress.getLocalHost) val userSession = Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "userA"), InetAddress.getLocalHost)
try { try {
// no quota set // 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 // Set default <user> quota config
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(10, true))) clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(10, true)))
assertEquals(10 * numFullQuotaWindows, quotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01) assertEquals(10 * numFullQuotaWindows, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01)
} finally { } finally {
quotaManager.shutdown() clientQuotaManager.shutdown()
} }
} }
@Test @Test
def testSetAndRemoveDefaultUserQuota(): Unit = { def testSetAndRemoveDefaultUserQuota(): Unit = {
// quotaTypesEnabled will be QuotaTypes.NoQuotas initially // 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, "") metrics, Produce, time, "")
try { try {
// no quota set yet, should not throttle // 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 // Set default <user> quota config
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(10, true))) clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(10, true)))
checkQuota(quotaManager, "userA", "client1", 10, 1000, true) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, true)
// Remove default <user> quota config, back to no quotas // Remove default <user> quota config, back to no quotas
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, None) clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, None)
checkQuota(quotaManager, "userA", "client1", Long.MaxValue, 1000, false) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, false)
} finally { } finally {
quotaManager.shutdown() clientQuotaManager.shutdown()
} }
} }
@Test @Test
def testSetAndRemoveUserQuota(): Unit = { def testSetAndRemoveUserQuota(): Unit = {
// quotaTypesEnabled will be QuotaTypes.NoQuotas initially // 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, "") metrics, Produce, time, "")
try { try {
// Set <user> quota config // Set <user> quota config
quotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(10, true))) clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(10, true)))
checkQuota(quotaManager, "userA", "client1", 10, 1000, true) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, true)
// Remove <user> quota config, back to no quotas // Remove <user> quota config, back to no quotas
quotaManager.updateQuota(Some("userA"), None, None, None) clientQuotaManager.updateQuota(Some("userA"), None, None, None)
checkQuota(quotaManager, "userA", "client1", Long.MaxValue, 1000, false) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, false)
} finally { } finally {
quotaManager.shutdown() clientQuotaManager.shutdown()
} }
} }
@Test @Test
def testSetAndRemoveUserClientQuota(): Unit = { def testSetAndRemoveUserClientQuota(): Unit = {
// quotaTypesEnabled will be QuotaTypes.NoQuotas initially // 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, "") metrics, Produce, time, "")
try { try {
// Set <user, client-id> quota config // Set <user, client-id> quota config
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10, true))) clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10, true)))
checkQuota(quotaManager, "userA", "client1", 10, 1000, true) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, true)
// Remove <user, client-id> quota config, back to no quotas // Remove <user, client-id> quota config, back to no quotas
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None) clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None)
checkQuota(quotaManager, "userA", "client1", Long.MaxValue, 1000, false) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, false)
} finally { } finally {
quotaManager.shutdown() clientQuotaManager.shutdown()
} }
} }
@Test @Test
def testQuotaConfigPrecedence(): Unit = { def testQuotaConfigPrecedence(): Unit = {
val quotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaBytesPerSecondDefault=Long.MaxValue), val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault=Long.MaxValue),
metrics, Produce, time, "") metrics, Produce, time, "")
try { try {
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, Some(new Quota(1000, true))) clientQuotaManager.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))) clientQuotaManager.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))) clientQuotaManager.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))) clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(4000, true)))
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(5000, true))) clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(5000, true)))
quotaManager.updateQuota(Some("userB"), None, None, Some(new Quota(6000, true))) clientQuotaManager.updateQuota(Some("userB"), None, None, Some(new Quota(6000, true)))
quotaManager.updateQuota(Some("userB"), Some("client1"), Some("client1"), Some(new Quota(7000, true))) clientQuotaManager.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))) clientQuotaManager.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))) clientQuotaManager.updateQuota(Some("userC"), None, None, Some(new Quota(10000, true)))
quotaManager.updateQuota(None, Some("client1"), Some("client1"), Some(new Quota(9000, 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(clientQuotaManager, "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(clientQuotaManager, "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(clientQuotaManager, "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, 0, false) // <user, client> is exclusive use, unaffected by other clients
checkQuota(quotaManager, "userB", "client1", 7000, 8000, true) checkQuota(clientQuotaManager, "userB", "client1", 7000, 8000, true)
checkQuota(quotaManager, "userB", "client2", 8000, 7000, false) // Default per-client quota for exclusive use of <user, client> checkQuota(clientQuotaManager, "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", "client3", 8000, 7000, false)
checkQuota(quotaManager, "userD", "client1", 3000, 3500, true) // Default <user, client> quota checkQuota(clientQuotaManager, "userD", "client1", 3000, 3500, true) // Default <user, client> quota
checkQuota(quotaManager, "userD", "client2", 3000, 2500, false) checkQuota(clientQuotaManager, "userD", "client2", 3000, 2500, false)
checkQuota(quotaManager, "userE", "client1", 3000, 2500, false) checkQuota(clientQuotaManager, "userE", "client1", 3000, 2500, false)
// Remove default <user, client> quota config, revert to <user> default // Remove default <user, client> quota config, revert to <user> default
quotaManager.updateQuota(Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), None) clientQuotaManager.updateQuota(Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), None)
checkQuota(quotaManager, "userD", "client1", 1000, 0, false) // Metrics tags changed, restart counter checkQuota(clientQuotaManager, "userD", "client1", 1000, 0, false) // Metrics tags changed, restart counter
checkQuota(quotaManager, "userE", "client4", 1000, 1500, true) checkQuota(clientQuotaManager, "userE", "client4", 1000, 1500, true)
checkQuota(quotaManager, "userF", "client4", 1000, 800, false) // Default <user> quota shared across clients of user checkQuota(clientQuotaManager, "userF", "client4", 1000, 800, false) // Default <user> quota shared across clients of user
checkQuota(quotaManager, "userF", "client5", 1000, 800, true) checkQuota(clientQuotaManager, "userF", "client5", 1000, 800, true)
// Remove default <user> quota config, revert to <client-id> default // Remove default <user> quota config, revert to <client-id> default
quotaManager.updateQuota(Some(ConfigEntityName.Default), None, None, None) clientQuotaManager.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(clientQuotaManager, "userF", "client4", 2000, 0, false) // Default <client-id> quota shared across client-id of all users
checkQuota(quotaManager, "userF", "client5", 2000, 0, false) checkQuota(clientQuotaManager, "userF", "client5", 2000, 0, false)
checkQuota(quotaManager, "userF", "client5", 2000, 2500, true) checkQuota(clientQuotaManager, "userF", "client5", 2000, 2500, true)
checkQuota(quotaManager, "userG", "client5", 2000, 0, true) checkQuota(clientQuotaManager, "userG", "client5", 2000, 0, true)
// Update quotas // Update quotas
quotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(8000, true))) clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(8000, true)))
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10000, true))) clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10000, true)))
checkQuota(quotaManager, "userA", "client2", 8000, 0, false) checkQuota(clientQuotaManager, "userA", "client2", 8000, 0, false)
checkQuota(quotaManager, "userA", "client2", 8000, 4500, true) // Throttled due to sum of new and earlier values checkQuota(clientQuotaManager, "userA", "client2", 8000, 4500, true) // Throttled due to sum of new and earlier values
checkQuota(quotaManager, "userA", "client1", 10000, 0, false) checkQuota(clientQuotaManager, "userA", "client1", 10000, 0, false)
checkQuota(quotaManager, "userA", "client1", 10000, 6000, true) checkQuota(clientQuotaManager, "userA", "client1", 10000, 6000, true)
quotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None) clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None)
checkQuota(quotaManager, "userA", "client6", 8000, 0, true) // Throttled due to shared user quota checkQuota(clientQuotaManager, "userA", "client6", 8000, 0, true) // Throttled due to shared user quota
quotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), Some(new Quota(11000, true))) clientQuotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), Some(new Quota(11000, true)))
checkQuota(quotaManager, "userA", "client6", 11000, 8500, false) checkQuota(clientQuotaManager, "userA", "client6", 11000, 8500, false)
quotaManager.updateQuota(Some("userA"), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(12000, true))) clientQuotaManager.updateQuota(Some("userA"), Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), Some(new Quota(12000, true)))
quotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), None) clientQuotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), None)
checkQuota(quotaManager, "userA", "client6", 12000, 4000, true) // Throttled due to sum of new and earlier values checkQuota(clientQuotaManager, "userA", "client6", 12000, 4000, true) // Throttled due to sum of new and earlier values
} finally { } finally {
quotaManager.shutdown() clientQuotaManager.shutdown()
} }
} }
@Test @Test
def testQuotaViolation(): Unit = { 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", "")) val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", "Produce", ""))
try { try {
/* We have 10 second windows. Make sure that there is no quota violation // We have 10 second windows. Make sure that there is no quota violation
* if we produce under the quota // if we produce under the quota
*/
for (_ <- 0 until 10) { for (_ <- 0 until 10) {
assertEquals(0, maybeRecord(clientMetrics, "ANONYMOUS", "unknown", 400)) assertEquals(0, maybeRecord(clientQuotaManager, "ANONYMOUS", "unknown", 400))
time.sleep(1000) time.sleep(1000)
} }
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt) 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 // (600 - quota)/quota*window-size = (600-500)/500*10.5 seconds = 2100
// 10.5 seconds because the last window is half complete // 10.5 seconds because the last window is half complete
time.sleep(500) time.sleep(500)
val sleepTime = maybeRecord(clientMetrics, "ANONYMOUS", "unknown", 2300) val throttleTime = maybeRecord(clientQuotaManager, "ANONYMOUS", "unknown", 2300)
assertEquals("Should be throttled", 2100, sleepTime) assertEquals("Should be throttled", 2100, throttleTime)
throttle(clientMetrics, "ANONYMOYUS", "unknown", sleepTime, callback) throttle(clientQuotaManager, "ANONYMOUS", "unknown", throttleTime, callback)
assertEquals(1, queueSizeMetric.metricValue.asInstanceOf[Double].toInt) assertEquals(1, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
// After a request is delayed, the callback cannot be triggered immediately // After a request is delayed, the callback cannot be triggered immediately
clientMetrics.throttledChannelReaper.doWork() clientQuotaManager.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()
assertEquals(0, numCallbacks) assertEquals(0, numCallbacks)
time.sleep(throttleTime) time.sleep(throttleTime)
// Callback can only be triggered after the delay time passes // Callback can only be triggered after the delay time passes
quotaManager.throttledChannelReaper.doWork() clientQuotaManager.throttledChannelReaper.doWork()
assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt) assertEquals(0, queueSizeMetric.metricValue.asInstanceOf[Double].toInt)
assertEquals(1, numCallbacks) assertEquals(1, numCallbacks)
// Could continue to see delays until the bursty sample disappears // 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) { for (_ <- 0 until 10) {
maybeRecord(clientQuotaManager, "ANONYMOUS", "unknown", 400)
time.sleep(1000) 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 { } finally {
quotaManager.shutdown() clientQuotaManager.shutdown()
} }
} }
@Test @Test
def testExpireThrottleTimeSensor(): Unit = { def testExpireThrottleTimeSensor(): Unit = {
val clientMetrics = new ClientQuotaManager(config, metrics, Produce, time, "") val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "")
try { try {
maybeRecord(clientMetrics, "ANONYMOUS", "client1", 100) maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100)
// remove the throttle time sensor // remove the throttle time sensor
metrics.removeSensor("ProduceThrottleTime-:client1") metrics.removeSensor("ProduceThrottleTime-:client1")
// should not throw an exception even if the throttle time sensor does not exist. // 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) assertTrue("Should be throttled", throttleTime > 0)
// the sensor should get recreated // the sensor should get recreated
val throttleTimeSensor = metrics.getSensor("ProduceThrottleTime-:client1") val throttleTimeSensor = metrics.getSensor("ProduceThrottleTime-:client1")
assertTrue("Throttle time sensor should exist", throttleTimeSensor != null) assertTrue("Throttle time sensor should exist", throttleTimeSensor != null)
assertTrue("Throttle time sensor should exist", throttleTimeSensor != null) assertTrue("Throttle time sensor should exist", throttleTimeSensor != null)
} finally { } finally {
clientMetrics.shutdown() clientQuotaManager.shutdown()
} }
} }
@Test @Test
def testExpireQuotaSensors(): Unit = { def testExpireQuotaSensors(): Unit = {
val clientMetrics = new ClientQuotaManager(config, metrics, Produce, time, "") val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "")
try { try {
maybeRecord(clientMetrics, "ANONYMOUS", "client1", 100) maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100)
// remove all the sensors // remove all the sensors
metrics.removeSensor("ProduceThrottleTime-:client1") metrics.removeSensor("ProduceThrottleTime-:client1")
metrics.removeSensor("Produce-ANONYMOUS:client1") metrics.removeSensor("Produce-ANONYMOUS:client1")
// should not throw an exception // 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) assertTrue("Should be throttled", throttleTime > 0)
// all the sensors should get recreated // all the sensors should get recreated
@ -497,16 +382,16 @@ class ClientQuotaManagerTest {
val byteRateSensor = metrics.getSensor("Produce-:client1") val byteRateSensor = metrics.getSensor("Produce-:client1")
assertTrue("Byte rate sensor should exist", byteRateSensor != null) assertTrue("Byte rate sensor should exist", byteRateSensor != null)
} finally { } finally {
clientMetrics.shutdown() clientQuotaManager.shutdown()
} }
} }
@Test @Test
def testClientIdNotSanitized(): Unit = { def testClientIdNotSanitized(): Unit = {
val clientMetrics = new ClientQuotaManager(config, metrics, Produce, time, "") val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "")
val clientId = "client@#$%" val clientId = "client@#$%"
try { 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 // The metrics should use the raw client ID, even if the reporters internally sanitize them
val throttleTimeSensor = metrics.getSensor("ProduceThrottleTime-:" + clientId) val throttleTimeSensor = metrics.getSensor("ProduceThrottleTime-:" + clientId)
@ -515,7 +400,7 @@ class ClientQuotaManagerTest {
val byteRateSensor = metrics.getSensor("Produce-:" + clientId) val byteRateSensor = metrics.getSensor("Produce-:" + clientId)
assertTrue("Byte rate sensor should exist", byteRateSensor != null) assertTrue("Byte rate sensor should exist", byteRateSensor != null)
} finally { } finally {
clientMetrics.shutdown() clientQuotaManager.shutdown()
} }
} }

View File

@ -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()
}
}
}

View File

@ -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])
}
}
}

View File

@ -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)
}
}

View File

@ -94,9 +94,10 @@ class KafkaApisTest {
private val metadataCache = new MetadataCache(brokerId) private val metadataCache = new MetadataCache(brokerId)
private val clientQuotaManager: ClientQuotaManager = EasyMock.createNiceMock(classOf[ClientQuotaManager]) private val clientQuotaManager: ClientQuotaManager = EasyMock.createNiceMock(classOf[ClientQuotaManager])
private val clientRequestQuotaManager: ClientRequestQuotaManager = EasyMock.createNiceMock(classOf[ClientRequestQuotaManager]) 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 replicaQuotaManager: ReplicationQuotaManager = EasyMock.createNiceMock(classOf[ReplicationQuotaManager])
private val quotas = QuotaManagers(clientQuotaManager, clientQuotaManager, clientRequestQuotaManager, 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 fetchManager: FetchManager = EasyMock.createNiceMock(classOf[FetchManager])
private val brokerTopicStats = new BrokerTopicStats private val brokerTopicStats = new BrokerTopicStats
private val clusterId = "clusterId" private val clusterId = "clusterId"

View File

@ -177,22 +177,16 @@ class RequestQuotaTest extends BaseRequestTest {
} }
private def throttleTimeMetricValueForQuotaType(clientId: String, quotaType: QuotaType): Double = { private def throttleTimeMetricValueForQuotaType(clientId: String, quotaType: QuotaType): Double = {
val metricName = leaderNode.metrics.metricName("throttle-time", val metricName = leaderNode.metrics.metricName("throttle-time", quotaType.toString,
quotaType.toString, "", "user", "", "client-id", clientId)
"",
"user", "",
"client-id", clientId)
val sensor = leaderNode.quotaManagers.request.getOrCreateQuotaSensors(session("ANONYMOUS"), val sensor = leaderNode.quotaManagers.request.getOrCreateQuotaSensors(session("ANONYMOUS"),
clientId).throttleTimeSensor clientId).throttleTimeSensor
metricValue(leaderNode.metrics.metrics.get(metricName), sensor) metricValue(leaderNode.metrics.metrics.get(metricName), sensor)
} }
private def requestTimeMetricValue(clientId: String): Double = { private def requestTimeMetricValue(clientId: String): Double = {
val metricName = leaderNode.metrics.metricName("request-time", val metricName = leaderNode.metrics.metricName("request-time", QuotaType.Request.toString,
QuotaType.Request.toString, "", "user", "", "client-id", clientId)
"",
"user", "",
"client-id", clientId)
val sensor = leaderNode.quotaManagers.request.getOrCreateQuotaSensors(session("ANONYMOUS"), val sensor = leaderNode.quotaManagers.request.getOrCreateQuotaSensors(session("ANONYMOUS"),
clientId).quotaSensor clientId).quotaSensor
metricValue(leaderNode.metrics.metrics.get(metricName), sensor) metricValue(leaderNode.metrics.metrics.get(metricName), sensor)

View File

@ -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 condition condition to check
* @param msg error message * @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. * 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)( 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() val startTime = System.currentTimeMillis()
while (true) { while (true) {
val result = compute val result = compute

View File

@ -25,6 +25,7 @@ import kafka.server.AdminManager;
import kafka.server.BrokerTopicStats; import kafka.server.BrokerTopicStats;
import kafka.server.ClientQuotaManager; import kafka.server.ClientQuotaManager;
import kafka.server.ClientRequestQuotaManager; import kafka.server.ClientRequestQuotaManager;
import kafka.server.ControllerMutationQuotaManager;
import kafka.server.FetchManager; import kafka.server.FetchManager;
import kafka.server.KafkaApis; import kafka.server.KafkaApis;
import kafka.server.KafkaConfig; import kafka.server.KafkaConfig;
@ -100,10 +101,11 @@ public class MetadataRequestBenchmark {
private MetadataCache metadataCache = new MetadataCache(brokerId); private MetadataCache metadataCache = new MetadataCache(brokerId);
private ClientQuotaManager clientQuotaManager = Mockito.mock(ClientQuotaManager.class); private ClientQuotaManager clientQuotaManager = Mockito.mock(ClientQuotaManager.class);
private ClientRequestQuotaManager clientRequestQuotaManager = Mockito.mock(ClientRequestQuotaManager.class); private ClientRequestQuotaManager clientRequestQuotaManager = Mockito.mock(ClientRequestQuotaManager.class);
private ControllerMutationQuotaManager controllerMutationQuotaManager = Mockito.mock(ControllerMutationQuotaManager.class);
private ReplicationQuotaManager replicaQuotaManager = Mockito.mock(ReplicationQuotaManager.class); private ReplicationQuotaManager replicaQuotaManager = Mockito.mock(ReplicationQuotaManager.class);
private QuotaFactory.QuotaManagers quotaManagers = new QuotaFactory.QuotaManagers(clientQuotaManager, private QuotaFactory.QuotaManagers quotaManagers = new QuotaFactory.QuotaManagers(clientQuotaManager,
clientQuotaManager, clientRequestQuotaManager, replicaQuotaManager, replicaQuotaManager, clientQuotaManager, clientRequestQuotaManager, controllerMutationQuotaManager, replicaQuotaManager,
replicaQuotaManager, Option.empty()); replicaQuotaManager, replicaQuotaManager, Option.empty());
private FetchManager fetchManager = Mockito.mock(FetchManager.class); private FetchManager fetchManager = Mockito.mock(FetchManager.class);
private BrokerTopicStats brokerTopicStats = new BrokerTopicStats(); private BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
private KafkaPrincipal principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "test-user"); private KafkaPrincipal principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "test-user");