From 9efd21225f9a81572e3067ebc0dab04d9344f604 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 9 Apr 2015 17:18:37 -0700 Subject: [PATCH 01/10] WIP: First patch for quotas. Changes are 1. Adding per-client throttle time and quota metrics in ClientQuotaMetrics.scala 2. Making changes in QuotaViolationException and Sensor to return delay time changes. 3. Added configuration needed so far for quotas in KafkaConfig. 4. Unit tests This is currently not being used anywhere in the code because I haven't yet figured out how to enforce delays i.e. purgatory vs delay queue. I'll have a better idea once I look at the new purgatory implementation. Hopefully, this smaller patch is easier to review. --- .../apache/kafka/common/metrics/MetricConfig.java | 20 +- .../org/apache/kafka/common/metrics/Quota.java | 24 +++ .../common/metrics/QuotaViolationException.java | 8 +- .../org/apache/kafka/common/metrics/Sensor.java | 26 ++- .../scala/kafka/server/ClientQuotaMetrics.scala | 204 +++++++++++++++++++++ core/src/main/scala/kafka/server/KafkaConfig.scala | 56 +++++- core/src/main/scala/kafka/server/KafkaServer.scala | 15 +- .../unit/kafka/server/ClientQuotaMetricsTest.scala | 106 +++++++++++ 8 files changed, 453 insertions(+), 6 deletions(-) create mode 100644 core/src/main/scala/kafka/server/ClientQuotaMetrics.scala create mode 100644 core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java index dfa1b0a..e45692b 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java @@ -28,6 +28,7 @@ public class MetricConfig { private long eventWindow; private long timeWindowMs; private TimeUnit unit; + private long quotaEnforcementDelayMs; public MetricConfig() { super(); @@ -36,6 +37,9 @@ public class MetricConfig { this.eventWindow = Long.MAX_VALUE; this.timeWindowMs = TimeUnit.MILLISECONDS.convert(30, TimeUnit.SECONDS); this.unit = TimeUnit.SECONDS; + // By default quota checking is disabled for the first window. + // This builds up of sufficient data initially before making quota enforcement decisions + this.quotaEnforcementDelayMs = timeWindowMs; } public Quota quota() { @@ -65,7 +69,21 @@ public class MetricConfig { return this; } - public int samples() { + /** + * Disables quota enforcement on this metric for this certain period of time + * after the metric is first created + */ + public MetricConfig quotaEnforcementDelay(long time, TimeUnit unit) { + this.quotaEnforcementDelayMs = TimeUnit.MILLISECONDS.convert(time, unit); + return this; + } + + public long quotaEnforcementDelayMs() { + return quotaEnforcementDelayMs; + } + + + public int samples() { return this.samples; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java index d82bb0c..007e4cf 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java @@ -49,4 +49,28 @@ public final class Quota { return (upper && value <= bound) || (!upper && value >= bound); } + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (int) this.bound; + result = prime * result + (this.upper ? 1 : 0); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Quota that = (Quota) obj; + if(that.bound != this.bound) + return false; + if(that.upper != this.upper) + return false; + return true; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java index a451e53..2e33dcf 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java @@ -24,9 +24,15 @@ import org.apache.kafka.common.KafkaException; public class QuotaViolationException extends KafkaException { private static final long serialVersionUID = 1L; + private final int delayTimeMs; public QuotaViolationException(String m) { - super(m); + this(m, 0); } + public QuotaViolationException(String m, int delayTimeMs) { + this.delayTimeMs = delayTimeMs; + } + + public int getDelayTimeMs() {return delayTimeMs;} } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index ca823fd..3be3dce 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -37,6 +37,7 @@ public final class Sensor { private final List metrics; private final MetricConfig config; private final Time time; + private final long creationTime; Sensor(Metrics registry, String name, Sensor[] parents, MetricConfig config, Time time) { super(); @@ -47,6 +48,7 @@ public final class Sensor { this.stats = new ArrayList(); this.config = config; this.time = time; + this.creationTime = time.milliseconds(); checkForest(new HashSet()); } @@ -112,13 +114,33 @@ public final class Sensor { if (config != null) { Quota quota = config.quota(); if (quota != null) { - if (!quota.acceptable(metric.value(timeMs))) - throw new QuotaViolationException(metric.metricName() + " is in violation of its quota of " + quota.bound()); + double value = metric.value(timeMs); + if(timeMs - this.creationTime < config.quotaEnforcementDelayMs()) + continue; + if (!quota.acceptable(value)) + { + throw new QuotaViolationException(String.format( + "(%s) is violating its quota of (%f) with value (%f)", + metric.metricName(), + quota.bound(), + value), + delayTime(metric, timeMs, quota, value, config)); + } } } } } + /* + * This calculates the amount of time needed to bring the metric within quota + * assuming that no new metrics are recorded + */ + private int delayTime(KafkaMetric metric, long timeMs, Quota quota, double metricValue, MetricConfig config) { + double difference = metricValue - quota.bound(); + double time = difference/quota.bound()*config.samples()*config.timeWindowMs(); + return (int) time; + } + /** * Register a compound statistic with this sensor with no config override */ diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala new file mode 100644 index 0000000..388549c --- /dev/null +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -0,0 +1,204 @@ +/** + * 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.message.{NoCompressionCodec, CompressionCodec} +import kafka.utils.Logging +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.metrics.stats.{Avg, Max, Rate} + +import scala.collection.mutable + +/** + * Configuration settings for quota management + * @param quotaDelayFactor The quota delay factor modifies any delay by a fixed multiplier (default 1.2) + * @param quotaEnforcementDelaySeconds The Sensor does not report quota violations for this amount of time after + * the sensor is created. This is meant to prevent throttling all clients + * upon server startup + * @param numQuotaSamples The number of sample to retain in memory + * @param quotaWindowSizeSeconds The time span of each sample + * + */ +case class ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond : Long = + ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond, + defaultConsumerQuotaBytesPerSecond : Long = + ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond, + producerQuotaOverrides : String = + ClientQuotaMetricsConfig.DefaultProducerQuotaOverrides, + consumerQuotaOverrides : String = + ClientQuotaMetricsConfig.DefaultConsumerQuotaOverrides, + quotaDelayFactor: Double = + ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, + quotaEnforcementDelaySeconds : Int = + ClientQuotaMetricsConfig.DefaultQuotaEnforcementDelaySeconds, + numQuotaSamples : Int = + ClientQuotaMetricsConfig.DefaultNumQuotaSamples, + quotaWindowSizeSeconds : Int = + ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) + +object ClientQuotaMetricsConfig { + val DefaultProducerQuotaBytesPerSecond = Long.MaxValue + val DefaultConsumerQuotaBytesPerSecond = Long.MaxValue + val DefaultProducerQuotaOverrides = "" + val DefaultConsumerQuotaOverrides = "" + val DefaultQuotaDelayFactor = 1.2; + val DefaultQuotaEnforcementDelaySeconds = 5; + // Always have 10 whole windows + 1 current window + val DefaultNumQuotaSamples = 11; + val DefaultQuotaWindowSizeSeconds = 1; +} + +object ClientQuotaMetrics { + private val ProducerSensorPrefix = "ProducerQuotaMetrics" + private val ConsumerSensorPrefix = "ConsumerQuotaMetrics" +} + +/** + * Helper class that records per-client metrics. It is also responsible for maintaining Quota usage statistics + * for all clients. + * @param config @KafkaConfig Configs for the Kafka Server + * @param metrics @Metrics Metrics instance + */ +class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, + private val metrics : Metrics) extends Logging { + private val producerOverriddenQuota = initQuotaMap(config.producerQuotaOverrides) + private val defaultBytesProducedQuota = Quota.lessThan(config.defaultProducerQuotaBytesPerSecond) + private val consumerOverriddenQuota = initQuotaMap(config.consumerQuotaOverrides) + private val defaultBytesConsumedQuota = Quota.lessThan(config.defaultConsumerQuotaBytesPerSecond) + + /** + * Records that a fetch request consumed some data + * @param clientId clientId that fetched the data + * @param bytes amount of data consumed in bytes + * @return Number of milliseconds to delay the response in case of Quota violation. + * Zero otherwise + */ + def recordBytesConsumed(clientId: String, bytes: Int) : Int = { + recordSensor(ClientQuotaMetrics.ConsumerSensorPrefix, clientId, consumerQuota(clientId), bytes) + } + /** + * Records that a produce request wrote some data + * @param clientId clientId that produced the data + * @param bytes amount of data written in bytes + * @return Number of milliseconds to delay the response in case of Quota violation. + * Zero otherwise + */ + def recordBytesProduced(clientId: String, bytes: Int) : Int = { + recordSensor(ClientQuotaMetrics.ProducerSensorPrefix, clientId, producerQuota(clientId), bytes) + } + + /** + * Returns the producer quota for the specified clientId + * @return + */ + def producerQuota(clientId : String) : Quota = { + if(producerOverriddenQuota.contains(clientId)) + producerOverriddenQuota(clientId) + else + defaultBytesProducedQuota + } + + /** + * Returns the consumer quota for the specified clientId + * @return + */ + def consumerQuota(clientId : String) : Quota = { + if(consumerOverriddenQuota.contains(clientId)) + consumerOverriddenQuota(clientId) + else + defaultBytesConsumedQuota + } + + private def recordSensor(sensorPrefix : String, + clientId : String, + quota : Quota, + value : Int) : Int = { + val sensors = getOrCreateQuotaSensors(sensorPrefix, clientId, quota) + var delayTime = 0.0 + try { + sensors._1.record(value) + } catch { + case qve : QuotaViolationException => + delayTime = qve.getDelayTimeMs()*config.quotaDelayFactor + sensors._2.record(delayTime) + logger.warn("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), qve) + } + delayTime.toInt + } + + private def getOrCreateQuotaSensors(sensorPrefix : String, + clientId : String, + quota : Quota) : (Sensor, Sensor) = { + val quotaSensorName = sensorPrefix + "-" + clientId + val throttleTimeSensorName = sensorPrefix + "ThrottleTime-" + clientId + var sensorVal = metrics.getSensor(quotaSensorName) + if (sensorVal == null) { + this.synchronized { + sensorVal = metrics.getSensor(quotaSensorName) + if (sensorVal == null) { + sensorVal = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota)) + sensorVal.add(new MetricName("byte-rate", + sensorPrefix, + "Tracking byte-rate per client", + "client-id", clientId), + new Rate()) + // create the throttle time sensor also + val throttleTimeSensor = metrics.sensor(throttleTimeSensorName) + throttleTimeSensor.add(new MetricName("throttle-time-max", + sensorPrefix, + "Tracking throttle-time per client", + "client-id", clientId), + new Max(), + new MetricConfig()) + throttleTimeSensor.add(new MetricName("throttle-time-avg", + sensorPrefix, + "Tracking throttle-time per client", + "client-id", clientId), + new Avg(), + new MetricConfig()) + } + } + } + (sensorVal, metrics.sensor(throttleTimeSensorName)) + } + + private def getQuotaMetricConfig(quota : Quota) : MetricConfig = { + new MetricConfig() + .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) + .samples(config.numQuotaSamples) + .quotaEnforcementDelay(config.quotaEnforcementDelaySeconds, TimeUnit.SECONDS) + .quota(quota) + } + + private def initQuotaMap(input : String) : mutable.Map[String, Quota] = { + val output = mutable.Map[String, Quota]() + for(entry <- input.split(",")) { + val trimmedEntry = entry.trim + if(!trimmedEntry.equals("")) { + val pair: Array[String] = trimmedEntry.split("=") + if (pair.length != 2) + throw new IllegalArgumentException("Incorrectly formatted override entry (%s). Format is k1=v1,k2=v2".format( + entry)) + output(pair(0)) = new Quota(pair(1).toDouble, true) + } + } + output + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9efa15c..ad753ca 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -123,6 +123,16 @@ object Defaults { val OffsetCommitTimeoutMs = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks + /** ********* Quota Configuration ***********/ + val ProducerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond + val ConsumerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond + val ProducerQuotaOverrides = ClientQuotaMetricsConfig.DefaultProducerQuotaOverrides + val ConsumerQuotaOverrides = ClientQuotaMetricsConfig.DefaultConsumerQuotaOverrides + val QuotaDelayFactor : Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor + val QuotaEnforcementDelaySeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaEnforcementDelaySeconds + val NumQuotaSamples : Int = ClientQuotaMetricsConfig.DefaultNumQuotaSamples + val QuotaWindowSizeSeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds + val DeleteTopicEnable = false val CompressionType = "producer" @@ -229,11 +239,19 @@ object KafkaConfig { val OffsetsRetentionCheckIntervalMsProp = "offsets.retention.check.interval.ms" val OffsetCommitTimeoutMsProp = "offsets.commit.timeout.ms" val OffsetCommitRequiredAcksProp = "offsets.commit.required.acks" + /** ********* Quota Configuration ***********/ + val ProducerQuotaDefaultBytesPerSecondProp = "quota.producer.default" + val ConsumerQuotaDefaultBytesPerSecondProp = "quota.consumer.default" + val ProducerQuotaOverridesProp = "quota.producer.overrides" + val ConsumerQuotaOverridesProp = "quota.consumer.overrides" + val QuotaDelayFactorProp = "quota.delay.factor" + val QuotaEnforcementDelaySecondsProp = "quota.delay.enforcement.seconds" + val NumQuotaSamplesProp = "quota.window.num" + val QuotaWindowSizeSecondsProp = "quota.window.size.seconds" val DeleteTopicEnableProp = "delete.topic.enable" val CompressionTypeProp = "compression.type" - /* Documentation */ /** ********* Zookeeper Configuration ***********/ val ZkConnectDoc = "Zookeeper host string" @@ -359,6 +377,18 @@ object KafkaConfig { val OffsetCommitTimeoutMsDoc = "Offset commit will be delayed until all replicas for the offsets topic receive the commit " + "or this timeout is reached. This is similar to the producer request timeout." val OffsetCommitRequiredAcksDoc = "The required acks before the commit can be accepted. In general, the default (-1) should not be overridden" + /** ********* Quota Configuration ***********/ + val ProducerQuotaDefaultBytesPerSecondDoc = "Any producer distinguished by clientId will get throttled if it produces more bytes than this value per-second" + val ConsumerQuotaDefaultBytesPerSecondDoc = "Any consumer distinguished by clientId/consumer group will get throttled if it fetches more bytes than this value per-second" + val ProducerQuotaOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default producer quota. " + + "Example: clientIdX=10485760,clientIdY=10485760" + val ConsumerQuotaOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default consumer quota. " + + "Example: clientIdX=10485760,clientIdY=10485760" + val QuotaDelayFactorDoc = "The quota delay factor modifies any delay by a fixed multiplier (default 1.2)" + val QuotaEnforcementDelaySecondsDoc = "The Sensor does not report quota violations for this amount of time after" + + " the sensor is created. This is meant to prevent throttling all clients upon server startup" + val NumQuotaSamplesDoc = "The number of samples to retain in memory" + val QuotaWindowSizeSecondsDoc = "The time span of each sample" val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + "('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + @@ -479,6 +509,16 @@ object KafkaConfig { .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) .define(CompressionTypeProp, STRING, Defaults.CompressionType, HIGH, CompressionTypeDoc) + + /** ********* Quota configuration ***********/ + .define(ProducerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ProducerQuotaDefaultBytesPerSecond, HIGH, ProducerQuotaDefaultBytesPerSecondDoc) + .define(ConsumerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ConsumerQuotaDefaultBytesPerSecond, HIGH, ConsumerQuotaDefaultBytesPerSecondDoc) + .define(ProducerQuotaOverridesProp, STRING, Defaults.ProducerQuotaOverrides, HIGH, ProducerQuotaOverridesDoc) + .define(ConsumerQuotaOverridesProp, STRING, Defaults.ConsumerQuotaOverrides, HIGH, ConsumerQuotaOverridesDoc) + .define(QuotaDelayFactorProp, DOUBLE, Defaults.QuotaDelayFactor, LOW, QuotaDelayFactorDoc) + .define(QuotaEnforcementDelaySecondsProp, INT, Defaults.QuotaEnforcementDelaySeconds, LOW, QuotaEnforcementDelaySecondsProp) + .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, LOW, NumQuotaSamplesDoc) + .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, LOW, QuotaWindowSizeSecondsDoc) } def configNames() = { @@ -597,6 +637,11 @@ object KafkaConfig { offsetsRetentionCheckIntervalMs = parsed.get(OffsetsRetentionCheckIntervalMsProp).asInstanceOf[Long], offsetCommitTimeoutMs = parsed.get(OffsetCommitTimeoutMsProp).asInstanceOf[Int], offsetCommitRequiredAcks = parsed.get(OffsetCommitRequiredAcksProp).asInstanceOf[Short], + /** ********* Quota configuration ***********/ + producerQuotaDefaultBytesPerSecond = parsed.get(ProducerQuotaDefaultBytesPerSecondProp).asInstanceOf[Long], + consumerQuotaDefaultBytesPerSecond = parsed.get(ConsumerQuotaDefaultBytesPerSecondProp).asInstanceOf[Long], + producerQuotaOverrides = parsed.get(ProducerQuotaOverridesProp).asInstanceOf[String], + consumerQuotaOverrides = parsed.get(ConsumerQuotaOverridesProp).asInstanceOf[String], deleteTopicEnable = parsed.get(DeleteTopicEnableProp).asInstanceOf[Boolean], compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String] ) @@ -741,6 +786,11 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val offsetCommitTimeoutMs: Int = Defaults.OffsetCommitTimeoutMs, val offsetCommitRequiredAcks: Short = Defaults.OffsetCommitRequiredAcks, + /** ********* Quota configuration ***********/ + val producerQuotaDefaultBytesPerSecond: Long = Defaults.ProducerQuotaDefaultBytesPerSecond, + val consumerQuotaDefaultBytesPerSecond: Long = Defaults.ConsumerQuotaDefaultBytesPerSecond, + val producerQuotaOverrides : String = Defaults.ProducerQuotaOverrides, + val consumerQuotaOverrides : String = Defaults.ConsumerQuotaOverrides, val deleteTopicEnable: Boolean = Defaults.DeleteTopicEnable, val compressionType: String = Defaults.CompressionType ) { @@ -962,6 +1012,10 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ props.put(OffsetsRetentionCheckIntervalMsProp, offsetsRetentionCheckIntervalMs.toString) props.put(OffsetCommitTimeoutMsProp, offsetCommitTimeoutMs.toString) props.put(OffsetCommitRequiredAcksProp, offsetCommitRequiredAcks.toString) + props.put(ProducerQuotaDefaultBytesPerSecondProp, producerQuotaDefaultBytesPerSecond.toString) + props.put(ConsumerQuotaDefaultBytesPerSecondProp, consumerQuotaDefaultBytesPerSecond.toString) + props.put(ProducerQuotaOverridesProp, producerQuotaOverrides.toString) + props.put(ConsumerQuotaOverridesProp, consumerQuotaOverrides.toString) props.put(DeleteTopicEnableProp, deleteTopicEnable.toString) props.put(CompressionTypeProp, compressionType.toString) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index b7d2a28..5d5aba2 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -27,6 +27,9 @@ import atomic.{AtomicInteger, AtomicBoolean} import java.io.File import collection.mutable +import org.apache.kafka.common.metrics.{Metrics, JmxReporter, MetricsReporter, MetricConfig} + +import scala.collection.{JavaConversions, mutable} import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} import kafka.cluster.{EndPoint, Broker} @@ -72,7 +75,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg val metadataCache: MetadataCache = new MetadataCache(config.brokerId) - + val metrics: Metrics = initMetrics() var zkClient: ZkClient = null val correlationId: AtomicInteger = new AtomicInteger(0) val brokerMetaPropsFile = "meta.properties" @@ -184,6 +187,16 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg } } + private def initMetrics() : Metrics = { + val jmxPrefix: String = "kafka.server" + val reporters: Seq[MetricsReporter] = mutable.Seq(new JmxReporter(jmxPrefix)) + info("Initiated metrics") + + new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + JavaConversions.seqAsJavaList(reporters), + new org.apache.kafka.common.utils.SystemTime) + } + private def initZk(): ZkClient = { info("Connecting to zookeeper on " + config.zkConnect) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala new file mode 100644 index 0000000..eea7a1e --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala @@ -0,0 +1,106 @@ +/** + * 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.Collections + +import kafka.utils.TestUtils +import org.apache.kafka.common.metrics.{Quota, JmxReporter, Metrics, MetricConfig} +import org.apache.kafka.common.utils.MockTime +import org.scalatest.junit.JUnit3Suite +import org.junit.{Test, Assert} + +class ClientQuotaMetricsTest extends JUnit3Suite { + private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + new MockTime) + private val config = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, + defaultConsumerQuotaBytesPerSecond = 1000, + producerQuotaOverrides = "p1=2000,p2=4000", + consumerQuotaOverrides = "c1=2000,c2=4000") + + @Test + def testQuotaParsing() { + val props = TestUtils.createBrokerConfig(1) + props.put(KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp, "500") + props.put(KafkaConfig.ProducerQuotaOverridesProp, "p1=2000,p2=4000") + props.put(KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp, "1000") + props.put(KafkaConfig.ConsumerQuotaOverridesProp, "c1=2000,c2=4000") + val clientMetrics = new ClientQuotaMetrics(config, metrics) + Assert.assertEquals("Default producer quota should be 500", + new Quota(500, true), clientMetrics.producerQuota("random-client-id")) + Assert.assertEquals("Default consumer quota should be 1000", + new Quota(1000, true), clientMetrics.consumerQuota("random-client-id")) + Assert.assertEquals("Should return the overridden value (2000)", + new Quota(2000, true), clientMetrics.producerQuota("p1")) + Assert.assertEquals("Should return the overridden value (4000)", + new Quota(4000, true), clientMetrics.producerQuota("p2")) + Assert.assertEquals("Should return the overridden value (2000)", + new Quota(2000, true), clientMetrics.consumerQuota("c1")) + Assert.assertEquals("Should return the overridden value (4000)", + new Quota(4000, true), clientMetrics.consumerQuota("c2")) + } + + @Test + def testProducerQuotaViolation() { + val time = new MockTime + val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + time) + val clientMetrics = new ClientQuotaMetrics(config, metrics) + /* We have 10 second windows. Make sure that there is no quota violation + * if we produce under the quota + */ + for(i <- 0 until 10) { + Assert.assertEquals(0, clientMetrics.recordBytesProduced("unknown", 400)) + time.sleep(1000) + } + + // Create a spike. + val sleepTime = clientMetrics.recordBytesProduced("unknown", 2000); + // 400*10 + 2000 = 6000/10 = 600 bytes per second. + // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200*1.2 = 2640 + Assert.assertEquals("Should be throttled", 2640, sleepTime) + time.sleep(sleepTime) + // At the end of sleep, the + Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.recordBytesProduced("unknown", 0)) + } + + @Test + def testConsumerQuotaViolation() { + val time = new MockTime + val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + time) + + val clientMetrics = new ClientQuotaMetrics(config, metrics) + for(i <- 0 until 10) { + Assert.assertEquals(0, clientMetrics.recordBytesConsumed("c1", 1600)) + time.sleep(1000) + } + + // Create a spike. + val sleepTime = clientMetrics.recordBytesConsumed("c1", 8000); + // 1600*10 + 8000 = 24000/10 = 2400 bytes per second. + // (2400 - quota)/quota*window-size = (2400-2000)/2000*11 seconds = 2200*1.2 = 2640 + Assert.assertEquals("Should be throttled", 2640, sleepTime) + time.sleep(sleepTime) + // At the end of sleep, the + Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.recordBytesConsumed("c1", 0)) + } + +} -- 1.7.12.4 From 5abb6516d28ae51ecdc5444b5718ac9828252a0f Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 9 Apr 2015 18:10:21 -0700 Subject: [PATCH 02/10] Added more testcases --- .../scala/kafka/server/ClientQuotaMetrics.scala | 11 ++- .../unit/kafka/server/ClientQuotaMetricsTest.scala | 88 ++++++++++++++++++---- 2 files changed, 80 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index 388549c..5ec7ba7 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -18,7 +18,6 @@ package kafka.server import java.util.concurrent.TimeUnit -import kafka.message.{NoCompressionCodec, CompressionCodec} import kafka.utils.Logging import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ @@ -58,11 +57,11 @@ object ClientQuotaMetricsConfig { val DefaultConsumerQuotaBytesPerSecond = Long.MaxValue val DefaultProducerQuotaOverrides = "" val DefaultConsumerQuotaOverrides = "" - val DefaultQuotaDelayFactor = 1.2; - val DefaultQuotaEnforcementDelaySeconds = 5; + val DefaultQuotaDelayFactor = 1.2 + val DefaultQuotaEnforcementDelaySeconds = 5 // Always have 10 whole windows + 1 current window - val DefaultNumQuotaSamples = 11; - val DefaultQuotaWindowSizeSeconds = 1; + val DefaultNumQuotaSamples = 11 + val DefaultQuotaWindowSizeSeconds = 1 } object ClientQuotaMetrics { @@ -136,7 +135,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, sensors._1.record(value) } catch { case qve : QuotaViolationException => - delayTime = qve.getDelayTimeMs()*config.quotaDelayFactor + delayTime = qve.getDelayTimeMs*config.quotaDelayFactor sensors._2.record(delayTime) logger.warn("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), qve) } diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala index eea7a1e..bcd86c1 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala @@ -18,8 +18,7 @@ package kafka.server import java.util.Collections -import kafka.utils.TestUtils -import org.apache.kafka.common.metrics.{Quota, JmxReporter, Metrics, MetricConfig} +import org.apache.kafka.common.metrics.{Quota, MetricConfig} import org.apache.kafka.common.utils.MockTime import org.scalatest.junit.JUnit3Suite import org.junit.{Test, Assert} @@ -28,18 +27,13 @@ class ClientQuotaMetricsTest extends JUnit3Suite { private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), Collections.emptyList(), new MockTime) - private val config = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, + private val config = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, defaultConsumerQuotaBytesPerSecond = 1000, - producerQuotaOverrides = "p1=2000,p2=4000", + producerQuotaOverrides = "p1=2000,p2=4000", consumerQuotaOverrides = "c1=2000,c2=4000") @Test def testQuotaParsing() { - val props = TestUtils.createBrokerConfig(1) - props.put(KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp, "500") - props.put(KafkaConfig.ProducerQuotaOverridesProp, "p1=2000,p2=4000") - props.put(KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp, "1000") - props.put(KafkaConfig.ConsumerQuotaOverridesProp, "c1=2000,c2=4000") val clientMetrics = new ClientQuotaMetrics(config, metrics) Assert.assertEquals("Default producer quota should be 500", new Quota(500, true), clientMetrics.producerQuota("random-client-id")) @@ -71,9 +65,9 @@ class ClientQuotaMetricsTest extends JUnit3Suite { } // Create a spike. - val sleepTime = clientMetrics.recordBytesProduced("unknown", 2000); + val sleepTime = clientMetrics.recordBytesProduced("unknown", 2000) // 400*10 + 2000 = 6000/10 = 600 bytes per second. - // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200*1.2 = 2640 + // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200*1.2 delayfactor = 2640 Assert.assertEquals("Should be throttled", 2640, sleepTime) time.sleep(sleepTime) // At the end of sleep, the @@ -82,6 +76,7 @@ class ClientQuotaMetricsTest extends JUnit3Suite { @Test def testConsumerQuotaViolation() { + val time = new MockTime val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), Collections.emptyList(), @@ -94,13 +89,80 @@ class ClientQuotaMetricsTest extends JUnit3Suite { } // Create a spike. - val sleepTime = clientMetrics.recordBytesConsumed("c1", 8000); + val sleepTime = clientMetrics.recordBytesConsumed("c1", 8000) // 1600*10 + 8000 = 24000/10 = 2400 bytes per second. - // (2400 - quota)/quota*window-size = (2400-2000)/2000*11 seconds = 2200*1.2 = 2640 + // (2400 - quota)/quota*window-size = (2400-2000)/2000*11 seconds = 2200*1.2 delayfactor = 2640 Assert.assertEquals("Should be throttled", 2640, sleepTime) time.sleep(sleepTime) // At the end of sleep, the Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.recordBytesConsumed("c1", 0)) } + @Test + def testOverrideParse() { + val time = new MockTime + val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + time) + // Case 1 - Default config + var testConfig = ClientQuotaMetricsConfig() + var clientMetrics = new ClientQuotaMetrics(testConfig, metrics) + Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond, true), + clientMetrics.producerQuota("p1")) + Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond, true), + clientMetrics.consumerQuota("p1")) + + // Case 2 - Empty override + testConfig = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, + defaultConsumerQuotaBytesPerSecond = 1000, + producerQuotaOverrides = "p1=2000,p2=4000,,", + consumerQuotaOverrides = "c1=2000,c2=4000") + + clientMetrics = new ClientQuotaMetrics(testConfig, metrics) + Assert.assertEquals(new Quota(2000, true), clientMetrics.producerQuota("p1")) + Assert.assertEquals(new Quota(4000, true), clientMetrics.producerQuota("p2")) + + // Case 3 - NumberFormatException for producer override + testConfig = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, + defaultConsumerQuotaBytesPerSecond = 1000, + producerQuotaOverrides = "p1=2000,p2=4000,p3=p4", + consumerQuotaOverrides = "c1=2000,c2=4000") + try { + clientMetrics = new ClientQuotaMetrics(testConfig, metrics) + Assert.fail("Should fail to parse invalid config " + testConfig.producerQuotaOverrides) + } + catch { + // Swallow. + case nfe : NumberFormatException => + } + + // Case 4 - NumberFormatException for consumer override + testConfig = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, + defaultConsumerQuotaBytesPerSecond = 1000, + producerQuotaOverrides = "p1=2000,p2=4000", + consumerQuotaOverrides = "c1=2000,c2=4000,c3=c4") + try { + clientMetrics = new ClientQuotaMetrics(testConfig, metrics) + Assert.fail("Should fail to parse invalid config " + testConfig.consumerQuotaOverrides) + } + catch { + // Swallow. + case nfe : NumberFormatException => + } + + // Case 5 - IllegalArgumentException for producer override + testConfig = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, + defaultConsumerQuotaBytesPerSecond = 1000, + producerQuotaOverrides = "p1=2000=3000", + consumerQuotaOverrides = "c1=2000,c2=4000") + try { + clientMetrics = new ClientQuotaMetrics(testConfig, metrics) + Assert.fail("Should fail to parse invalid config " + testConfig.producerQuotaOverrides) + } + catch { + // Swallow. + case nfe : IllegalArgumentException => + } + + } } -- 1.7.12.4 From bf4639c80424824760e216bde22e6732e80d8099 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Fri, 10 Apr 2015 17:24:06 -0700 Subject: [PATCH 03/10] Some locking changes for reading/creating the sensors --- .../scala/kafka/server/ClientQuotaMetrics.scala | 96 +++++++++++++++------- 1 file changed, 68 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index 5ec7ba7..c4ea832 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -22,6 +22,9 @@ import kafka.utils.Logging import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ import org.apache.kafka.common.metrics.stats.{Avg, Max, Rate} +import java.util.concurrent.locks.ReentrantReadWriteLock +import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock +import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock import scala.collection.mutable @@ -36,21 +39,21 @@ import scala.collection.mutable * */ case class ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond : Long = - ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond, + ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond, defaultConsumerQuotaBytesPerSecond : Long = - ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond, + ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond, producerQuotaOverrides : String = - ClientQuotaMetricsConfig.DefaultProducerQuotaOverrides, + ClientQuotaMetricsConfig.DefaultProducerQuotaOverrides, consumerQuotaOverrides : String = - ClientQuotaMetricsConfig.DefaultConsumerQuotaOverrides, + ClientQuotaMetricsConfig.DefaultConsumerQuotaOverrides, quotaDelayFactor: Double = - ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, + ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, quotaEnforcementDelaySeconds : Int = - ClientQuotaMetricsConfig.DefaultQuotaEnforcementDelaySeconds, + ClientQuotaMetricsConfig.DefaultQuotaEnforcementDelaySeconds, numQuotaSamples : Int = - ClientQuotaMetricsConfig.DefaultNumQuotaSamples, + ClientQuotaMetricsConfig.DefaultNumQuotaSamples, quotaWindowSizeSeconds : Int = - ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) + ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) object ClientQuotaMetricsConfig { val DefaultProducerQuotaBytesPerSecond = Long.MaxValue @@ -81,6 +84,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, private val defaultBytesProducedQuota = Quota.lessThan(config.defaultProducerQuotaBytesPerSecond) private val consumerOverriddenQuota = initQuotaMap(config.consumerQuotaOverrides) private val defaultBytesConsumedQuota = Quota.lessThan(config.defaultConsumerQuotaBytesPerSecond) + private val lock = new ReentrantReadWriteLock() /** * Records that a fetch request consumed some data @@ -100,7 +104,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, * Zero otherwise */ def recordBytesProduced(clientId: String, bytes: Int) : Int = { - recordSensor(ClientQuotaMetrics.ProducerSensorPrefix, clientId, producerQuota(clientId), bytes) + recordSensor(ClientQuotaMetrics.ProducerSensorPrefix, clientId, producerQuota(clientId), bytes) } /** @@ -142,40 +146,76 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, delayTime.toInt } + /* + * 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 + */ private def getOrCreateQuotaSensors(sensorPrefix : String, clientId : String, quota : Quota) : (Sensor, Sensor) = { + + // Names of the sensors to access val quotaSensorName = sensorPrefix + "-" + clientId val throttleTimeSensorName = sensorPrefix + "ThrottleTime-" + clientId - var sensorVal = metrics.getSensor(quotaSensorName) - if (sensorVal == null) { - this.synchronized { - sensorVal = metrics.getSensor(quotaSensorName) - if (sensorVal == null) { - sensorVal = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota)) - sensorVal.add(new MetricName("byte-rate", - sensorPrefix, - "Tracking byte-rate per client", - "client-id", clientId), - new Rate()) + var quotaSensor : Sensor = null + var throttleTimeSensor : Sensor = null + + /* Acquire the read lock to fetch the sensors. It is safe to call getSensor from multiple threads. + * The read lock allows a thread to create a sensor in isolation. The thread creating the sensor + * will acquire the write lock and prevent the sensors from being read while they are being created. + * It should be sufficient to simply check if the sensor is null without acquiring a read lock but the + * sensor being present doesn't mean that it is fully initialized i.e. all the Metrics may not have been added. + * This read lock waits until the writer thread has released it's lock i.e. fully initialized the sensor + * at which point it is safe to read + */ + lock.readLock().lock() + try { + quotaSensor = metrics.getSensor(quotaSensorName) + throttleTimeSensor = metrics.getSensor(throttleTimeSensorName) + } + finally { + lock.readLock().unlock() + } + + /* If the sensor is null, try to create it else return the created sensor + * Also if quota sensor is null, the throttle time sensor must be null + */ + if(quotaSensor == null) { + /* Acquire a write lock because the sensor may not have been created and we only want one thread to create it. + * Note that multiple threads may acquire the write lock if they all see a null sensor initially + * In this case, the writer checks the sensor after acquiring the lock again. + * This is safe from Double Checked Locking because the references are read + * after acquiring read locks and hence they cannot see a partially published reference + */ + lock.writeLock().lock() + try { + quotaSensor = metrics.getSensor(quotaSensorName) + if(quotaSensor == null) { // create the throttle time sensor also - val throttleTimeSensor = metrics.sensor(throttleTimeSensorName) + throttleTimeSensor = metrics.sensor(throttleTimeSensorName) throttleTimeSensor.add(new MetricName("throttle-time-max", sensorPrefix, "Tracking throttle-time per client", - "client-id", clientId), - new Max(), - new MetricConfig()) + "client-id", + clientId), new Max(), new MetricConfig()) throttleTimeSensor.add(new MetricName("throttle-time-avg", sensorPrefix, "Tracking throttle-time per client", - "client-id", clientId), - new Avg(), - new MetricConfig()) + "client-id", + clientId), new Avg(), new MetricConfig()) + quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota)) + quotaSensor.add(new MetricName("byte-rate", + sensorPrefix, + "Tracking byte-rate per client", + "client-id", + clientId), new Rate()) } + } finally { + lock.writeLock().unlock() } } - (sensorVal, metrics.sensor(throttleTimeSensorName)) + // return the read or created sensors + (quotaSensor, throttleTimeSensor) } private def getQuotaMetricConfig(quota : Quota) : MetricConfig = { -- 1.7.12.4 From 2c32f2b389f6d4b961090ce0f03f5b486643d024 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 21 Apr 2015 12:20:32 -0700 Subject: [PATCH 04/10] WIP patch --- .../apache/kafka/common/metrics/MetricConfig.java | 15 +++++----- .../common/metrics/QuotaViolationException.java | 4 ++- .../org/apache/kafka/common/metrics/Sensor.java | 2 +- .../scala/kafka/server/ClientQuotaMetrics.scala | 35 ++++++++++++---------- .../main/scala/kafka/server/ReplicaManager.scala | 1 - 5 files changed, 31 insertions(+), 26 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java index e45692b..db349e5 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java @@ -28,7 +28,7 @@ public class MetricConfig { private long eventWindow; private long timeWindowMs; private TimeUnit unit; - private long quotaEnforcementDelayMs; + private long quotaEnforcementBlackoutMs; public MetricConfig() { super(); @@ -39,7 +39,7 @@ public class MetricConfig { this.unit = TimeUnit.SECONDS; // By default quota checking is disabled for the first window. // This builds up of sufficient data initially before making quota enforcement decisions - this.quotaEnforcementDelayMs = timeWindowMs; + this.quotaEnforcementBlackoutMs = timeWindowMs; } public Quota quota() { @@ -73,17 +73,16 @@ public class MetricConfig { * Disables quota enforcement on this metric for this certain period of time * after the metric is first created */ - public MetricConfig quotaEnforcementDelay(long time, TimeUnit unit) { - this.quotaEnforcementDelayMs = TimeUnit.MILLISECONDS.convert(time, unit); + public MetricConfig quotaEnforcementBlackout(long time, TimeUnit unit) { + this.quotaEnforcementBlackoutMs = TimeUnit.MILLISECONDS.convert(time, unit); return this; } - public long quotaEnforcementDelayMs() { - return quotaEnforcementDelayMs; + public long quotaEnforcementBlackoutMs() { + return quotaEnforcementBlackoutMs; } - - public int samples() { + public int samples() { return this.samples; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java index 2e33dcf..9c929ea 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java @@ -24,15 +24,17 @@ import org.apache.kafka.common.KafkaException; public class QuotaViolationException extends KafkaException { private static final long serialVersionUID = 1L; - private final int delayTimeMs; + private int delayTimeMs; public QuotaViolationException(String m) { this(m, 0); } public QuotaViolationException(String m, int delayTimeMs) { + super(m); this.delayTimeMs = delayTimeMs; } + public void setDelayTimeMs(int delayTimeMs) { this.delayTimeMs = delayTimeMs; } public int getDelayTimeMs() {return delayTimeMs;} } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index 3be3dce..565f101 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -115,7 +115,7 @@ public final class Sensor { Quota quota = config.quota(); if (quota != null) { double value = metric.value(timeMs); - if(timeMs - this.creationTime < config.quotaEnforcementDelayMs()) + if(timeMs - this.creationTime < config.quotaEnforcementBlackoutMs()) continue; if (!quota.acceptable(value)) { diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index c4ea832..aebe93f 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -39,21 +39,21 @@ import scala.collection.mutable * */ case class ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond : Long = - ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond, + ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond, defaultConsumerQuotaBytesPerSecond : Long = - ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond, + ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond, producerQuotaOverrides : String = - ClientQuotaMetricsConfig.DefaultProducerQuotaOverrides, + ClientQuotaMetricsConfig.DefaultProducerQuotaOverrides, consumerQuotaOverrides : String = - ClientQuotaMetricsConfig.DefaultConsumerQuotaOverrides, + ClientQuotaMetricsConfig.DefaultConsumerQuotaOverrides, quotaDelayFactor: Double = - ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, + ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, quotaEnforcementDelaySeconds : Int = - ClientQuotaMetricsConfig.DefaultQuotaEnforcementDelaySeconds, + ClientQuotaMetricsConfig.DefaultQuotaEnforcementDelaySeconds, numQuotaSamples : Int = - ClientQuotaMetricsConfig.DefaultNumQuotaSamples, + ClientQuotaMetricsConfig.DefaultNumQuotaSamples, quotaWindowSizeSeconds : Int = - ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) + ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) object ClientQuotaMetricsConfig { val DefaultProducerQuotaBytesPerSecond = Long.MaxValue @@ -86,6 +86,10 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, private val defaultBytesConsumedQuota = Quota.lessThan(config.defaultConsumerQuotaBytesPerSecond) private val lock = new ReentrantReadWriteLock() + def record(key : String, clientId: String, value : Int) = { + recordSensor(ClientQuotaMetrics.ConsumerSensorPrefix, clientId, consumerQuota(clientId), value) + } + /** * Records that a fetch request consumed some data * @param clientId clientId that fetched the data @@ -93,7 +97,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, * @return Number of milliseconds to delay the response in case of Quota violation. * Zero otherwise */ - def recordBytesConsumed(clientId: String, bytes: Int) : Int = { + def recordBytesConsumed(clientId: String, bytes: Int) = { recordSensor(ClientQuotaMetrics.ConsumerSensorPrefix, clientId, consumerQuota(clientId), bytes) } /** @@ -103,7 +107,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, * @return Number of milliseconds to delay the response in case of Quota violation. * Zero otherwise */ - def recordBytesProduced(clientId: String, bytes: Int) : Int = { + def recordBytesProduced(clientId: String, bytes: Int) = { recordSensor(ClientQuotaMetrics.ProducerSensorPrefix, clientId, producerQuota(clientId), bytes) } @@ -111,7 +115,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, * Returns the producer quota for the specified clientId * @return */ - def producerQuota(clientId : String) : Quota = { + private[server] def producerQuota(clientId : String) : Quota = { if(producerOverriddenQuota.contains(clientId)) producerOverriddenQuota(clientId) else @@ -122,7 +126,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, * Returns the consumer quota for the specified clientId * @return */ - def consumerQuota(clientId : String) : Quota = { + private[server] def consumerQuota(clientId : String) : Quota = { if(consumerOverriddenQuota.contains(clientId)) consumerOverriddenQuota(clientId) else @@ -142,6 +146,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, delayTime = qve.getDelayTimeMs*config.quotaDelayFactor sensors._2.record(delayTime) logger.warn("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), qve) + throw qve } delayTime.toInt } @@ -197,12 +202,12 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, sensorPrefix, "Tracking throttle-time per client", "client-id", - clientId), new Max(), new MetricConfig()) + clientId), new Max()) throttleTimeSensor.add(new MetricName("throttle-time-avg", sensorPrefix, "Tracking throttle-time per client", "client-id", - clientId), new Avg(), new MetricConfig()) + clientId), new Avg()) quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota)) quotaSensor.add(new MetricName("byte-rate", sensorPrefix, @@ -222,7 +227,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, new MetricConfig() .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) .samples(config.numQuotaSamples) - .quotaEnforcementDelay(config.quotaEnforcementDelaySeconds, TimeUnit.SECONDS) + .quotaEnforcementBlackout(config.quotaEnforcementDelaySeconds, TimeUnit.SECONDS) .quota(quota) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 59c9bc3..46d549e 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -279,7 +279,6 @@ class ReplicaManager(val config: KafkaConfig, internalTopicsAllowed: Boolean, messagesPerPartition: Map[TopicAndPartition, MessageSet], responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { - if (isValidRequiredAcks(requiredAcks)) { val sTime = SystemTime.milliseconds -- 1.7.12.4 From 13c88d757e07c10b0a47488571d3cbd061f61c86 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 21 Apr 2015 12:27:52 -0700 Subject: [PATCH 05/10] Sample usage in ReplicaManager --- .../scala/kafka/server/ClientQuotaMetrics2.scala | 200 +++++++++++++++++++++ .../main/scala/kafka/server/ReplicaManager.scala | 21 +++ .../kafka/server/ClientQuotaMetricsTest2.scala | 121 +++++++++++++ 3 files changed, 342 insertions(+) create mode 100644 core/src/main/scala/kafka/server/ClientQuotaMetrics2.scala create mode 100644 core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest2.scala diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics2.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics2.scala new file mode 100644 index 0000000..bfaa694 --- /dev/null +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics2.scala @@ -0,0 +1,200 @@ +/** + * 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.utils.Logging +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.metrics.stats.{Avg, Max, Rate} +import java.util.concurrent.locks.ReentrantReadWriteLock + +import scala.collection.mutable + +/** + * Configuration settings for quota management + * @param quotaDelayFactor The quota delay factor modifies any delay by a fixed multiplier (default 1.2) + * @param quotaEnforcementDelaySeconds The Sensor does not report quota violations for this amount of time after + * the sensor is created. This is meant to prevent throttling all clients + * upon server startup + * @param numQuotaSamples The number of sample to retain in memory + * @param quotaWindowSizeSeconds The time span of each sample + * + */ +case class ClientQuotaMetricsConfig2(defaultQuotaPerSecond : Long = + ClientQuotaMetricsConfig2.DefaultQuotaPerSecond, + quotaOverrides : String = + ClientQuotaMetricsConfig2.QuotaOverrides, + quotaDelayFactor: Double = + ClientQuotaMetricsConfig2.DefaultQuotaDelayFactor, + quotaEnforcementDelaySeconds : Int = + ClientQuotaMetricsConfig2.DefaultQuotaEnforcementDelaySeconds, + numQuotaSamples : Int = + ClientQuotaMetricsConfig2.DefaultNumQuotaSamples, + quotaWindowSizeSeconds : Int = + ClientQuotaMetricsConfig2.DefaultQuotaWindowSizeSeconds) + +object ClientQuotaMetricsConfig2 { + val DefaultQuotaPerSecond = Long.MaxValue + val QuotaOverrides = "" + val DefaultQuotaDelayFactor = 1.2 + val DefaultQuotaEnforcementDelaySeconds = 5 + // Always have 10 whole windows + 1 current window + val DefaultNumQuotaSamples = 11 + val DefaultQuotaWindowSizeSeconds = 1 +} + +/** + * Helper class that records per-client metrics. It is also responsible for maintaining Quota usage statistics + * for all clients. + * @param config @KafkaConfig Configs for the Kafka Server + * @param metrics @Metrics Metrics instance + */ +class ClientQuotaMetrics2(private val config : ClientQuotaMetricsConfig2, + private val metrics : Metrics, + private val apiKey : String) extends Logging { + private val overriddenQuota = initQuotaMap(config.quotaOverrides) + private val defaultQuota = Quota.lessThan(config.defaultQuotaPerSecond) + private val lock = new ReentrantReadWriteLock() + + /** + * Records that a produce request wrote some data + * @param clientId clientId that produced the data + * @param value amount of data written in bytes + * @return Number of milliseconds to delay the response in case of Quota violation. + * Zero otherwise + */ + def record(clientId: String, value : Int) = { + val sensors = getOrCreateQuotaSensors(clientId) + var delayTime = 0.0 + try { + sensors._1.record(value) + } catch { + case qve : QuotaViolationException => + delayTime = qve.getDelayTimeMs*config.quotaDelayFactor + qve.setDelayTimeMs(delayTime.toInt) + sensors._2.record(delayTime) + logger.warn("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), qve) + throw qve + } + } + + /** + * Returns the consumer quota for the specified clientId + * @return + */ + private[server] def quota(clientId : String) : Quota = { + if(overriddenQuota.contains(clientId)) + overriddenQuota(clientId) + else + defaultQuota + } + + /* + * 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 + */ + private def getOrCreateQuotaSensors(clientId : String) : (Sensor, Sensor) = { + + // Names of the sensors to access + val quotaSensorName = apiKey + "-" + clientId + val throttleTimeSensorName = apiKey + "ThrottleTime-" + clientId + var quotaSensor : Sensor = null + var throttleTimeSensor : Sensor = null + + /* Acquire the read lock to fetch the sensors. It is safe to call getSensor from multiple threads. + * The read lock allows a thread to create a sensor in isolation. The thread creating the sensor + * will acquire the write lock and prevent the sensors from being read while they are being created. + * It should be sufficient to simply check if the sensor is null without acquiring a read lock but the + * sensor being present doesn't mean that it is fully initialized i.e. all the Metrics may not have been added. + * This read lock waits until the writer thread has released it's lock i.e. fully initialized the sensor + * at which point it is safe to read + */ + lock.readLock().lock() + try { + quotaSensor = metrics.getSensor(quotaSensorName) + throttleTimeSensor = metrics.getSensor(throttleTimeSensorName) + } + finally { + lock.readLock().unlock() + } + + /* If the sensor is null, try to create it else return the created sensor + * Also if quota sensor is null, the throttle time sensor must be null + */ + if(quotaSensor == null) { + /* Acquire a write lock because the sensor may not have been created and we only want one thread to create it. + * Note that multiple threads may acquire the write lock if they all see a null sensor initially + * In this case, the writer checks the sensor after acquiring the lock again. + * This is safe from Double Checked Locking because the references are read + * after acquiring read locks and hence they cannot see a partially published reference + */ + lock.writeLock().lock() + try { + quotaSensor = metrics.getSensor(quotaSensorName) + if(quotaSensor == null) { + // create the throttle time sensor also + throttleTimeSensor = metrics.sensor(throttleTimeSensorName) + throttleTimeSensor.add(new MetricName("throttle-time-max", + apiKey, + "Tracking throttle-time per client", + "client-id", + clientId), new Max()) + throttleTimeSensor.add(new MetricName("throttle-time-avg", + apiKey, + "Tracking throttle-time per client", + "client-id", + clientId), new Avg()) + quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota(clientId))) + quotaSensor.add(new MetricName("byte-rate", + apiKey, + "Tracking byte-rate per client", + "client-id", + clientId), new Rate()) + } + } finally { + lock.writeLock().unlock() + } + } + // return the read or created sensors + (quotaSensor, throttleTimeSensor) + } + + private def getQuotaMetricConfig(quota : Quota) : MetricConfig = { + new MetricConfig() + .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) + .samples(config.numQuotaSamples) + .quotaEnforcementBlackout(config.quotaEnforcementDelaySeconds, TimeUnit.SECONDS) + .quota(quota) + } + + private def initQuotaMap(input : String) : mutable.Map[String, Quota] = { + val output = mutable.Map[String, Quota]() + for(entry <- input.split(",")) { + val trimmedEntry = entry.trim + if(!trimmedEntry.equals("")) { + val pair: Array[String] = trimmedEntry.split("=") + if (pair.length != 2) + throw new IllegalArgumentException("Incorrectly formatted override entry (%s). Format is k1=v1,k2=v2".format( + entry)) + output(pair(0)) = new Quota(pair(1).toDouble, true) + } + } + output + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 46d549e..6b6281e 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -111,6 +111,14 @@ class ReplicaManager(val config: KafkaConfig, val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch]( purgatoryName = "Fetch", config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) + // Method 1 + val clientMetrics = new ClientQuotaMetrics(null, null) + + // Method 2 + val producerMetrics = new ClientQuotaMetrics2(null, null, "producer") + val consumerMetrics = new ClientQuotaMetrics2(null, null, "consumer") + + newGauge( "LeaderCount", new Gauge[Int] { @@ -279,6 +287,13 @@ class ReplicaManager(val config: KafkaConfig, internalTopicsAllowed: Boolean, messagesPerPartition: Map[TopicAndPartition, MessageSet], responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { + + // Example of method 1 + clientMetrics.recordBytesProduced("xx", 1) + + // Method 2 + producerMetrics.record("xx", 1) + if (isValidRequiredAcks(requiredAcks)) { val sTime = SystemTime.milliseconds @@ -418,6 +433,12 @@ class ReplicaManager(val config: KafkaConfig, fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + // Example of method 1 + clientMetrics.recordBytesConsumed("xx", 1) + + // Method 2 + consumerMetrics.record("xx", 1) + val isFromFollower = replicaId >= 0 val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest2.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest2.scala new file mode 100644 index 0000000..5f3c5e1 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest2.scala @@ -0,0 +1,121 @@ +/** + * 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.Collections + +import org.apache.kafka.common.metrics.{QuotaViolationException, Quota, MetricConfig} +import org.apache.kafka.common.utils.MockTime +import org.scalatest.junit.JUnit3Suite +import org.junit.{Test, Assert} + +class ClientQuotaMetricsTest2 extends JUnit3Suite { + private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + new MockTime) + private val config = ClientQuotaMetricsConfig2(defaultQuotaPerSecond = 500, + quotaOverrides = "p1=2000,p2=4000") + + @Test + def testQuotaParsing() { + val clientMetrics = new ClientQuotaMetrics2(config, metrics, "producer") + Assert.assertEquals("Default producer quota should be 500", + new Quota(500, true), clientMetrics.quota("random-client-id")) + Assert.assertEquals("Should return the overridden value (2000)", + new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals("Should return the overridden value (4000)", + new Quota(4000, true), clientMetrics.quota("p2")) + } + + @Test + def testQuotaViolation() { + val time = new MockTime + val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + time) + val clientMetrics = new ClientQuotaMetrics2(config, metrics, "producer") + /* We have 10 second windows. Make sure that there is no quota violation + * if we produce under the quota + */ + for(i <- 0 until 10) { + clientMetrics.record("unknown", 400) + time.sleep(1000) + } + + // Create a spike. + var sleepTime = 0 + try { + clientMetrics.record("unknown", 2000) + } + catch { + case qve : QuotaViolationException => + sleepTime = qve.getDelayTimeMs + } + // 400*10 + 2000 = 6000/10 = 600 bytes per second. + // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200*1.2 delayfactor = 2640 + Assert.assertEquals("Should be throttled", 2640, sleepTime) + time.sleep(sleepTime) + // At the end of sleep, the + //Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.record("unknown", 0)) + } + + @Test + def testOverrideParse() { + val time = new MockTime + val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + time) + // Case 1 - Default config + var testConfig = ClientQuotaMetricsConfig2() + var clientMetrics = new ClientQuotaMetrics2(testConfig, metrics, "consumer") + Assert.assertEquals(new Quota(ClientQuotaMetricsConfig2.DefaultQuotaPerSecond, true), + clientMetrics.quota("p1")) + + // Case 2 - Empty override + testConfig = ClientQuotaMetricsConfig2(defaultQuotaPerSecond = 500, + quotaOverrides = "p1=2000,p2=4000,,") + + clientMetrics = new ClientQuotaMetrics2(testConfig, metrics, "consumer") + Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) + + // Case 3 - NumberFormatException for override + testConfig = ClientQuotaMetricsConfig2(defaultQuotaPerSecond = 500, + quotaOverrides = "p1=2000,p2=4000,p3=p4") + try { + clientMetrics = new ClientQuotaMetrics2(testConfig, metrics, "consumer") + Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) + } + catch { + // Swallow. + case nfe : NumberFormatException => + } + + // Case 4 - IllegalArgumentException for override + testConfig = ClientQuotaMetricsConfig2(defaultQuotaPerSecond = 500, + quotaOverrides = "p1=2000=3000") + try { + clientMetrics = new ClientQuotaMetrics2(testConfig, metrics, "producer") + Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) + } + catch { + // Swallow. + case nfe : IllegalArgumentException => + } + + } +} -- 1.7.12.4 From c78f7cd16dde436da276e04a6de933067bbbd337 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 4 May 2015 10:49:59 -0700 Subject: [PATCH 06/10] Updated patch for quotas. This patch does the following: 1. Add per-client metrics for both producer and consumers 2. Add configuration for quotas 3. Compute delay times in the metrics package and return the delay times in QuotaViolationException 4. Add a DelayQueue in KafkaApi's that can be used to throttle any type of request. Implemented request throttling for produce and fetch requests. 5. Added unit and integration test cases. I've not yet added integration testcases testing the consumer delays.. will update the patch once those are ready --- .../apache/kafka/common/metrics/MetricConfig.java | 12 +- .../org/apache/kafka/common/metrics/Quota.java | 46 ++--- .../common/metrics/QuotaViolationException.java | 2 +- .../org/apache/kafka/common/metrics/Sensor.java | 28 +-- .../org/apache/kafka/common/utils/MockTime.java | 43 +++++ .../org/apache/kafka/common/utils/MockTime.java | 43 ----- .../scala/kafka/server/ClientQuotaMetrics.scala | 125 +++++-------- .../scala/kafka/server/ClientQuotaMetrics2.scala | 200 --------------------- core/src/main/scala/kafka/server/KafkaApis.scala | 134 ++++++++++++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 38 ++-- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- .../main/scala/kafka/server/ReplicaManager.scala | 24 +-- .../main/scala/kafka/server/ThrottledRequest.scala | 22 +++ .../scala/kafka/utils/ShutdownableThread.scala | 3 + .../scala/integration/kafka/api/QuotasTest.scala | 144 +++++++++++++++ .../unit/kafka/server/ClientQuotaMetricsTest.scala | 121 ++++--------- .../kafka/server/ClientQuotaMetricsTest2.scala | 121 ------------- .../kafka/server/KafkaConfigConfigDefTest.scala | 17 +- .../server/ThrottledRequestExpirationTest.scala | 70 ++++++++ 19 files changed, 561 insertions(+), 634 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/utils/MockTime.java delete mode 100644 clients/src/test/java/org/apache/kafka/common/utils/MockTime.java delete mode 100644 core/src/main/scala/kafka/server/ClientQuotaMetrics2.scala create mode 100644 core/src/main/scala/kafka/server/ThrottledRequest.scala create mode 100644 core/src/test/scala/integration/kafka/api/QuotasTest.scala delete mode 100644 core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest2.scala create mode 100644 core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java index db349e5..76a6f4f 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java @@ -37,9 +37,7 @@ public class MetricConfig { this.eventWindow = Long.MAX_VALUE; this.timeWindowMs = TimeUnit.MILLISECONDS.convert(30, TimeUnit.SECONDS); this.unit = TimeUnit.SECONDS; - // By default quota checking is disabled for the first window. - // This builds up of sufficient data initially before making quota enforcement decisions - this.quotaEnforcementBlackoutMs = timeWindowMs; + this.quotaEnforcementBlackoutMs = 0; } public Quota quota() { @@ -74,13 +72,11 @@ public class MetricConfig { * after the metric is first created */ public MetricConfig quotaEnforcementBlackout(long time, TimeUnit unit) { - this.quotaEnforcementBlackoutMs = TimeUnit.MILLISECONDS.convert(time, unit); - return this; + this.quotaEnforcementBlackoutMs = TimeUnit.MILLISECONDS.convert(time, unit); + return this; } - public long quotaEnforcementBlackoutMs() { - return quotaEnforcementBlackoutMs; - } + public long quotaEnforcementBlackoutMs() { return quotaEnforcementBlackoutMs; } public int samples() { return this.samples; diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java index 007e4cf..e87edb5 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java @@ -49,28 +49,28 @@ public final class Quota { return (upper && value <= bound) || (!upper && value >= bound); } - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + (int) this.bound; - result = prime * result + (this.upper ? 1 : 0); - return result; - } + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (int) this.bound; + result = prime * result + (this.upper ? 1 : 0); + return result; + } - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Quota that = (Quota) obj; - if(that.bound != this.bound) - return false; - if(that.upper != this.upper) - return false; - return true; - } + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Quota that = (Quota) obj; + if (that.bound != this.bound) + return false; + if (that.upper != this.upper) + return false; + return true; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java index 9c929ea..dcc415c 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java @@ -36,5 +36,5 @@ public class QuotaViolationException extends KafkaException { } public void setDelayTimeMs(int delayTimeMs) { this.delayTimeMs = delayTimeMs; } - public int getDelayTimeMs() {return delayTimeMs;} + public int getDelayTimeMs() { return delayTimeMs; } } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index 565f101..73e9965 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -114,17 +114,17 @@ public final class Sensor { if (config != null) { Quota quota = config.quota(); if (quota != null) { - double value = metric.value(timeMs); - if(timeMs - this.creationTime < config.quotaEnforcementBlackoutMs()) - continue; - if (!quota.acceptable(value)) - { - throw new QuotaViolationException(String.format( - "(%s) is violating its quota of (%f) with value (%f)", - metric.metricName(), - quota.bound(), - value), - delayTime(metric, timeMs, quota, value, config)); + double value = metric.value(timeMs); + if (timeMs - this.creationTime < config.quotaEnforcementBlackoutMs()) + continue; + + if (!quota.acceptable(value)) { + throw new QuotaViolationException(String.format( + "(%s) is violating its quota of (%f) with value (%f)", + metric.metricName(), + quota.bound(), + value), + delayTime(metric, timeMs, quota, value, config)); } } } @@ -136,9 +136,9 @@ public final class Sensor { * assuming that no new metrics are recorded */ private int delayTime(KafkaMetric metric, long timeMs, Quota quota, double metricValue, MetricConfig config) { - double difference = metricValue - quota.bound(); - double time = difference/quota.bound()*config.samples()*config.timeWindowMs(); - return (int) time; + double difference = metricValue - quota.bound(); + double time = difference / quota.bound() * config.samples() * config.timeWindowMs(); + return (int) time; } /** diff --git a/clients/src/main/java/org/apache/kafka/common/utils/MockTime.java b/clients/src/main/java/org/apache/kafka/common/utils/MockTime.java new file mode 100644 index 0000000..eb7fcf0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/utils/MockTime.java @@ -0,0 +1,43 @@ +/** + * 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.utils; + +import java.util.concurrent.TimeUnit; + +/** + * A clock that you can manually advance by calling sleep + */ +public class MockTime implements Time { + + private long nanos = 0; + + public MockTime() { + this.nanos = System.nanoTime(); + } + + @Override + public long milliseconds() { + return TimeUnit.MILLISECONDS.convert(this.nanos, TimeUnit.NANOSECONDS); + } + + @Override + public long nanoseconds() { + return nanos; + } + + @Override + public void sleep(long ms) { + this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java deleted file mode 100644 index eb7fcf0..0000000 --- a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java +++ /dev/null @@ -1,43 +0,0 @@ -/** - * 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.utils; - -import java.util.concurrent.TimeUnit; - -/** - * A clock that you can manually advance by calling sleep - */ -public class MockTime implements Time { - - private long nanos = 0; - - public MockTime() { - this.nanos = System.nanoTime(); - } - - @Override - public long milliseconds() { - return TimeUnit.MILLISECONDS.convert(this.nanos, TimeUnit.NANOSECONDS); - } - - @Override - public long nanoseconds() { - return nanos; - } - - @Override - public void sleep(long ms) { - this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS); - } - -} diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index aebe93f..e2c0e74 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -23,53 +23,43 @@ import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ import org.apache.kafka.common.metrics.stats.{Avg, Max, Rate} import java.util.concurrent.locks.ReentrantReadWriteLock -import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock -import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock import scala.collection.mutable /** * Configuration settings for quota management + * @param defaultQuotaPerSecond The default quota allocated to any client + * @param quotaOverrides The comma separated overrides per client. "c1=X,c2=Y" * @param quotaDelayFactor The quota delay factor modifies any delay by a fixed multiplier (default 1.2) - * @param quotaEnforcementDelaySeconds The Sensor does not report quota violations for this amount of time after + * @param quotaEnforcementBlackoutSeconds The Sensor does not report quota violations for this amount of time after * the sensor is created. This is meant to prevent throttling all clients * upon server startup * @param numQuotaSamples The number of sample to retain in memory * @param quotaWindowSizeSeconds The time span of each sample * */ -case class ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond : Long = - ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond, - defaultConsumerQuotaBytesPerSecond : Long = - ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond, - producerQuotaOverrides : String = - ClientQuotaMetricsConfig.DefaultProducerQuotaOverrides, - consumerQuotaOverrides : String = - ClientQuotaMetricsConfig.DefaultConsumerQuotaOverrides, +case class ClientQuotaMetricsConfig(defaultQuotaPerSecond : Long = + ClientQuotaMetricsConfig.DefaultQuotaPerSecond, + quotaOverrides : String = + ClientQuotaMetricsConfig.QuotaOverrides, quotaDelayFactor: Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, - quotaEnforcementDelaySeconds : Int = - ClientQuotaMetricsConfig.DefaultQuotaEnforcementDelaySeconds, + quotaEnforcementBlackoutSeconds : Int = + ClientQuotaMetricsConfig.DefaultQuotaEnforcementBlackoutSeconds, numQuotaSamples : Int = ClientQuotaMetricsConfig.DefaultNumQuotaSamples, quotaWindowSizeSeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) object ClientQuotaMetricsConfig { - val DefaultProducerQuotaBytesPerSecond = Long.MaxValue - val DefaultConsumerQuotaBytesPerSecond = Long.MaxValue - val DefaultProducerQuotaOverrides = "" - val DefaultConsumerQuotaOverrides = "" + val DefaultQuotaPerSecond = Long.MaxValue + val QuotaOverrides = "" val DefaultQuotaDelayFactor = 1.2 - val DefaultQuotaEnforcementDelaySeconds = 5 + val DefaultQuotaEnforcementBlackoutSeconds = 5 // Always have 10 whole windows + 1 current window val DefaultNumQuotaSamples = 11 val DefaultQuotaWindowSizeSeconds = 1 -} - -object ClientQuotaMetrics { - private val ProducerSensorPrefix = "ProducerQuotaMetrics" - private val ConsumerSensorPrefix = "ConsumerQuotaMetrics" + val MaxThrottleTimeSeconds = 30 } /** @@ -79,89 +69,54 @@ object ClientQuotaMetrics { * @param metrics @Metrics Metrics instance */ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, - private val metrics : Metrics) extends Logging { - private val producerOverriddenQuota = initQuotaMap(config.producerQuotaOverrides) - private val defaultBytesProducedQuota = Quota.lessThan(config.defaultProducerQuotaBytesPerSecond) - private val consumerOverriddenQuota = initQuotaMap(config.consumerQuotaOverrides) - private val defaultBytesConsumedQuota = Quota.lessThan(config.defaultConsumerQuotaBytesPerSecond) + private val metrics : Metrics, + private val apiKey : String) extends Logging { + private val overriddenQuota = initQuotaMap(config.quotaOverrides) + private val defaultQuota = Quota.lessThan(config.defaultQuotaPerSecond) private val lock = new ReentrantReadWriteLock() - def record(key : String, clientId: String, value : Int) = { - recordSensor(ClientQuotaMetrics.ConsumerSensorPrefix, clientId, consumerQuota(clientId), value) - } - - /** - * Records that a fetch request consumed some data - * @param clientId clientId that fetched the data - * @param bytes amount of data consumed in bytes - * @return Number of milliseconds to delay the response in case of Quota violation. - * Zero otherwise - */ - def recordBytesConsumed(clientId: String, bytes: Int) = { - recordSensor(ClientQuotaMetrics.ConsumerSensorPrefix, clientId, consumerQuota(clientId), bytes) - } /** * Records that a produce request wrote some data * @param clientId clientId that produced the data - * @param bytes amount of data written in bytes + * @param value amount of data written in bytes * @return Number of milliseconds to delay the response in case of Quota violation. * Zero otherwise */ - def recordBytesProduced(clientId: String, bytes: Int) = { - recordSensor(ClientQuotaMetrics.ProducerSensorPrefix, clientId, producerQuota(clientId), bytes) - } - - /** - * Returns the producer quota for the specified clientId - * @return - */ - private[server] def producerQuota(clientId : String) : Quota = { - if(producerOverriddenQuota.contains(clientId)) - producerOverriddenQuota(clientId) - else - defaultBytesProducedQuota - } - - /** - * Returns the consumer quota for the specified clientId - * @return - */ - private[server] def consumerQuota(clientId : String) : Quota = { - if(consumerOverriddenQuota.contains(clientId)) - consumerOverriddenQuota(clientId) - else - defaultBytesConsumedQuota - } - - private def recordSensor(sensorPrefix : String, - clientId : String, - quota : Quota, - value : Int) : Int = { - val sensors = getOrCreateQuotaSensors(sensorPrefix, clientId, quota) + def record(clientId: String, value : Int) = { + val sensors = getOrCreateQuotaSensors(clientId) var delayTime = 0.0 try { sensors._1.record(value) } catch { case qve : QuotaViolationException => delayTime = qve.getDelayTimeMs*config.quotaDelayFactor + qve.setDelayTimeMs(delayTime.toInt) sensors._2.record(delayTime) logger.warn("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), qve) throw qve } - delayTime.toInt + } + + /** + * Returns the consumer quota for the specified clientId + * @return + */ + private[server] def quota(clientId : String) : Quota = { + if(overriddenQuota.contains(clientId)) + overriddenQuota(clientId) + else + defaultQuota } /* * 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 */ - private def getOrCreateQuotaSensors(sensorPrefix : String, - clientId : String, - quota : Quota) : (Sensor, Sensor) = { + private def getOrCreateQuotaSensors(clientId : String) : (Sensor, Sensor) = { // Names of the sensors to access - val quotaSensorName = sensorPrefix + "-" + clientId - val throttleTimeSensorName = sensorPrefix + "ThrottleTime-" + clientId + val quotaSensorName = apiKey + "-" + clientId + val throttleTimeSensorName = apiKey + "ThrottleTime-" + clientId var quotaSensor : Sensor = null var throttleTimeSensor : Sensor = null @@ -199,18 +154,18 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, // create the throttle time sensor also throttleTimeSensor = metrics.sensor(throttleTimeSensorName) throttleTimeSensor.add(new MetricName("throttle-time-max", - sensorPrefix, + apiKey, "Tracking throttle-time per client", "client-id", clientId), new Max()) throttleTimeSensor.add(new MetricName("throttle-time-avg", - sensorPrefix, + apiKey, "Tracking throttle-time per client", "client-id", clientId), new Avg()) - quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota)) + quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota(clientId))) quotaSensor.add(new MetricName("byte-rate", - sensorPrefix, + apiKey, "Tracking byte-rate per client", "client-id", clientId), new Rate()) @@ -227,7 +182,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, new MetricConfig() .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) .samples(config.numQuotaSamples) - .quotaEnforcementBlackout(config.quotaEnforcementDelaySeconds, TimeUnit.SECONDS) + .quotaEnforcementBlackout(config.quotaEnforcementBlackoutSeconds, TimeUnit.SECONDS) .quota(quota) } diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics2.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics2.scala deleted file mode 100644 index bfaa694..0000000 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics2.scala +++ /dev/null @@ -1,200 +0,0 @@ -/** - * 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.utils.Logging -import org.apache.kafka.common.MetricName -import org.apache.kafka.common.metrics._ -import org.apache.kafka.common.metrics.stats.{Avg, Max, Rate} -import java.util.concurrent.locks.ReentrantReadWriteLock - -import scala.collection.mutable - -/** - * Configuration settings for quota management - * @param quotaDelayFactor The quota delay factor modifies any delay by a fixed multiplier (default 1.2) - * @param quotaEnforcementDelaySeconds The Sensor does not report quota violations for this amount of time after - * the sensor is created. This is meant to prevent throttling all clients - * upon server startup - * @param numQuotaSamples The number of sample to retain in memory - * @param quotaWindowSizeSeconds The time span of each sample - * - */ -case class ClientQuotaMetricsConfig2(defaultQuotaPerSecond : Long = - ClientQuotaMetricsConfig2.DefaultQuotaPerSecond, - quotaOverrides : String = - ClientQuotaMetricsConfig2.QuotaOverrides, - quotaDelayFactor: Double = - ClientQuotaMetricsConfig2.DefaultQuotaDelayFactor, - quotaEnforcementDelaySeconds : Int = - ClientQuotaMetricsConfig2.DefaultQuotaEnforcementDelaySeconds, - numQuotaSamples : Int = - ClientQuotaMetricsConfig2.DefaultNumQuotaSamples, - quotaWindowSizeSeconds : Int = - ClientQuotaMetricsConfig2.DefaultQuotaWindowSizeSeconds) - -object ClientQuotaMetricsConfig2 { - val DefaultQuotaPerSecond = Long.MaxValue - val QuotaOverrides = "" - val DefaultQuotaDelayFactor = 1.2 - val DefaultQuotaEnforcementDelaySeconds = 5 - // Always have 10 whole windows + 1 current window - val DefaultNumQuotaSamples = 11 - val DefaultQuotaWindowSizeSeconds = 1 -} - -/** - * Helper class that records per-client metrics. It is also responsible for maintaining Quota usage statistics - * for all clients. - * @param config @KafkaConfig Configs for the Kafka Server - * @param metrics @Metrics Metrics instance - */ -class ClientQuotaMetrics2(private val config : ClientQuotaMetricsConfig2, - private val metrics : Metrics, - private val apiKey : String) extends Logging { - private val overriddenQuota = initQuotaMap(config.quotaOverrides) - private val defaultQuota = Quota.lessThan(config.defaultQuotaPerSecond) - private val lock = new ReentrantReadWriteLock() - - /** - * Records that a produce request wrote some data - * @param clientId clientId that produced the data - * @param value amount of data written in bytes - * @return Number of milliseconds to delay the response in case of Quota violation. - * Zero otherwise - */ - def record(clientId: String, value : Int) = { - val sensors = getOrCreateQuotaSensors(clientId) - var delayTime = 0.0 - try { - sensors._1.record(value) - } catch { - case qve : QuotaViolationException => - delayTime = qve.getDelayTimeMs*config.quotaDelayFactor - qve.setDelayTimeMs(delayTime.toInt) - sensors._2.record(delayTime) - logger.warn("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), qve) - throw qve - } - } - - /** - * Returns the consumer quota for the specified clientId - * @return - */ - private[server] def quota(clientId : String) : Quota = { - if(overriddenQuota.contains(clientId)) - overriddenQuota(clientId) - else - defaultQuota - } - - /* - * 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 - */ - private def getOrCreateQuotaSensors(clientId : String) : (Sensor, Sensor) = { - - // Names of the sensors to access - val quotaSensorName = apiKey + "-" + clientId - val throttleTimeSensorName = apiKey + "ThrottleTime-" + clientId - var quotaSensor : Sensor = null - var throttleTimeSensor : Sensor = null - - /* Acquire the read lock to fetch the sensors. It is safe to call getSensor from multiple threads. - * The read lock allows a thread to create a sensor in isolation. The thread creating the sensor - * will acquire the write lock and prevent the sensors from being read while they are being created. - * It should be sufficient to simply check if the sensor is null without acquiring a read lock but the - * sensor being present doesn't mean that it is fully initialized i.e. all the Metrics may not have been added. - * This read lock waits until the writer thread has released it's lock i.e. fully initialized the sensor - * at which point it is safe to read - */ - lock.readLock().lock() - try { - quotaSensor = metrics.getSensor(quotaSensorName) - throttleTimeSensor = metrics.getSensor(throttleTimeSensorName) - } - finally { - lock.readLock().unlock() - } - - /* If the sensor is null, try to create it else return the created sensor - * Also if quota sensor is null, the throttle time sensor must be null - */ - if(quotaSensor == null) { - /* Acquire a write lock because the sensor may not have been created and we only want one thread to create it. - * Note that multiple threads may acquire the write lock if they all see a null sensor initially - * In this case, the writer checks the sensor after acquiring the lock again. - * This is safe from Double Checked Locking because the references are read - * after acquiring read locks and hence they cannot see a partially published reference - */ - lock.writeLock().lock() - try { - quotaSensor = metrics.getSensor(quotaSensorName) - if(quotaSensor == null) { - // create the throttle time sensor also - throttleTimeSensor = metrics.sensor(throttleTimeSensorName) - throttleTimeSensor.add(new MetricName("throttle-time-max", - apiKey, - "Tracking throttle-time per client", - "client-id", - clientId), new Max()) - throttleTimeSensor.add(new MetricName("throttle-time-avg", - apiKey, - "Tracking throttle-time per client", - "client-id", - clientId), new Avg()) - quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota(clientId))) - quotaSensor.add(new MetricName("byte-rate", - apiKey, - "Tracking byte-rate per client", - "client-id", - clientId), new Rate()) - } - } finally { - lock.writeLock().unlock() - } - } - // return the read or created sensors - (quotaSensor, throttleTimeSensor) - } - - private def getQuotaMetricConfig(quota : Quota) : MetricConfig = { - new MetricConfig() - .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) - .samples(config.numQuotaSamples) - .quotaEnforcementBlackout(config.quotaEnforcementDelaySeconds, TimeUnit.SECONDS) - .quota(quota) - } - - private def initQuotaMap(input : String) : mutable.Map[String, Quota] = { - val output = mutable.Map[String, Quota]() - for(entry <- input.split(",")) { - val trimmedEntry = entry.trim - if(!trimmedEntry.equals("")) { - val pair: Array[String] = trimmedEntry.split("=") - if (pair.length != 2) - throw new IllegalArgumentException("Incorrectly formatted override entry (%s). Format is k1=v1,k2=v2".format( - entry)) - output(pair(0)) = new Quota(pair(1).toDouble, true) - } - } - output - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 417960d..6bf4b04 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,6 +17,11 @@ package kafka.server + +import java.util.concurrent.{TimeUnit, DelayQueue} + +import kafka.utils._ +import org.apache.kafka.common.metrics.{QuotaViolationException, Metrics} import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.requests.{JoinGroupResponse, JoinGroupRequest, HeartbeatRequest, HeartbeatResponse, ResponseHeader} import org.apache.kafka.common.TopicPartition @@ -45,9 +50,28 @@ class KafkaApis(val requestChannel: RequestChannel, val zkClient: ZkClient, val brokerId: Int, val config: KafkaConfig, - val metadataCache: MetadataCache) extends Logging { + val metadataCache: MetadataCache, + val metrics: Metrics) extends Logging { this.logIdent = "[KafkaApi-%d] ".format(brokerId) + // Store all the quota managers for each type of request + private val quotaManagers = instantiateQuotaManagers(config) + private val delayQueue = new DelayQueue[ThrottledRequest]() + private val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) + throttledRequestReaper.start() + + class ThrottledRequestReaper(delayQueue: DelayQueue[ThrottledRequest]) extends ShutdownableThread( + "ThrottledRequestReaper-%d".format(brokerId), false) { + + override def doWork(): Unit = { + val response : ThrottledRequest = delayQueue.poll(1, TimeUnit.SECONDS) + if(response != null) { + trace("Response throttled for: " + response.delayTimeMs + " ms") + //System.out.println("Response throttled for: " + response.delayTimeMs + " ms") + response.execute() + } + } + } /** * Top-level method that handles all requests and multiplexes to the right api @@ -227,11 +251,36 @@ class KafkaApis(val requestChannel: RequestChannel, } } + def throttleOrRespond(clientId : String, key : String, value: Int, callback: => Unit) = { + var throttleTime = 0 + quotaManagers.get(key) match { + case Some(quotaManager) => + try { + quotaManager.record(clientId, value) + + } + catch { + case qve : QuotaViolationException => + throttleTime = qve.getDelayTimeMs + //System.out.println("Request throttled for " + throttleTime + " ms") + } + case None => + warn("Cannot throttle Api key " + key) + } + if(throttleTime > 0) { + delayQueue.add(new ThrottledRequest(SystemTime, throttleTime, callback)) + } + else { + callback + } + } + /** * Handle a produce request */ def handleProducerRequest(request: RequestChannel.Request) { val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] + val numBytesAppended = produceRequest.sizeInBytes // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { @@ -247,21 +296,36 @@ class KafkaApis(val requestChannel: RequestChannel, } } - if (produceRequest.requiredAcks == 0) { - // no operation needed if producer request.required.acks = 0; however, if there is any error in handling - // the request, since no response is expected by the producer, the server will close socket server so that - // the producer client will know that some error has happened and will refresh its metadata - if (errorInResponse) { - info("Close connection due to error handling produce request with correlation id %d from client id %s with ack=0" - .format(produceRequest.correlationId, produceRequest.clientId)) - requestChannel.closeConnection(request.processor, request) - } else { - requestChannel.noOperation(request.processor, request) + def produceResponseCallback { + if (produceRequest.requiredAcks == 0) + { + // no operation needed if producer request.required.acks = 0; however, if there is any error in handling + // the request, since no response is expected by the producer, the server will close socket server so that + // the producer client will know that some error has happened and will refresh its metadata + if (errorInResponse) + { + info( + "Close connection due to error handling produce request with correlation id %d from client id %s with ack=0".format( + produceRequest.correlationId, + produceRequest.clientId)) + requestChannel.closeConnection(request.processor, request) + } + else + { + requestChannel.noOperation(request.processor, request) + } + } + else + { + val response = ProducerResponse(produceRequest.correlationId, responseStatus) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(response))) } - } else { - val response = ProducerResponse(produceRequest.correlationId, responseStatus) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } + + throttleOrRespond(produceRequest.clientId, + RequestKeys.nameForKey(RequestKeys.ProduceKey), + numBytesAppended, + produceResponseCallback) } // only allow appending to internal topic partitions @@ -287,6 +351,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] + System.out.println("Fetching " + fetchRequest.clientId) // the callback for sending a fetch response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { @@ -298,14 +363,19 @@ class KafkaApis(val requestChannel: RequestChannel, .format(fetchRequest.correlationId, fetchRequest.clientId, topicAndPartition, ErrorMapping.exceptionNameFor(data.error))) } - // record the bytes out metrics only when the response is being sent BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesOutRate.mark(data.messages.sizeInBytes) BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) } val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) + def fetchResponseCallback { + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) + } + throttleOrRespond(fetchRequest.clientId, + RequestKeys.nameForKey(RequestKeys.FetchKey), + response.sizeInBytes, + fetchResponseCallback) } // call the replica manager to fetch messages from the local replica @@ -584,9 +654,37 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseCallback) } + private def instantiateQuotaManagers(cfg : KafkaConfig): Map[String, ClientQuotaMetrics] = { + val producerQuotaManagerCfg = ClientQuotaMetricsConfig( + defaultQuotaPerSecond = cfg.producerQuotaDefaultBytesPerSecond, + quotaOverrides = cfg.producerQuotaOverrides, + quotaDelayFactor = cfg.quotaDelayFactor, + quotaEnforcementBlackoutSeconds = cfg.quotaEnforcementBlackoutSeconds, + numQuotaSamples = cfg.numQuotaSamples, + quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds + ) + + val consumerQuotaManagerCfg = ClientQuotaMetricsConfig( + defaultQuotaPerSecond = cfg.consumerQuotaDefaultBytesPerSecond, + quotaOverrides = cfg.consumerQuotaOverrides, + quotaDelayFactor = cfg.quotaDelayFactor, + quotaEnforcementBlackoutSeconds = cfg.quotaEnforcementBlackoutSeconds, + numQuotaSamples = cfg.numQuotaSamples, + quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds + ) + + val quotaManagers = Map[String, ClientQuotaMetrics]( + RequestKeys.nameForKey(RequestKeys.ProduceKey) -> + new ClientQuotaMetrics(producerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.ProduceKey)), + RequestKeys.nameForKey(RequestKeys.FetchKey) -> + new ClientQuotaMetrics(consumerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.FetchKey)) + ) + quotaManagers + } + def close() { - // TODO currently closing the API is an no-op since the API no longer maintain any modules - // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer + throttledRequestReaper.shutdown() + throttledRequestReaper.awaitShutdown() debug("Shut down complete.") } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index ad753ca..c7242ef 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -124,12 +124,12 @@ object Defaults { val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks /** ********* Quota Configuration ***********/ - val ProducerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond - val ConsumerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond - val ProducerQuotaOverrides = ClientQuotaMetricsConfig.DefaultProducerQuotaOverrides - val ConsumerQuotaOverrides = ClientQuotaMetricsConfig.DefaultConsumerQuotaOverrides + val ProducerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaPerSecond + val ConsumerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaPerSecond + val ProducerQuotaOverrides = ClientQuotaMetricsConfig.QuotaOverrides + val ConsumerQuotaOverrides = ClientQuotaMetricsConfig.QuotaOverrides val QuotaDelayFactor : Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor - val QuotaEnforcementDelaySeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaEnforcementDelaySeconds + val QuotaEnforcementBlackoutSeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaEnforcementBlackoutSeconds val NumQuotaSamples : Int = ClientQuotaMetricsConfig.DefaultNumQuotaSamples val QuotaWindowSizeSeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds @@ -245,7 +245,7 @@ object KafkaConfig { val ProducerQuotaOverridesProp = "quota.producer.overrides" val ConsumerQuotaOverridesProp = "quota.consumer.overrides" val QuotaDelayFactorProp = "quota.delay.factor" - val QuotaEnforcementDelaySecondsProp = "quota.delay.enforcement.seconds" + val QuotaEnforcementBlackoutSecondsProp = "quota.blackout.enforcement.seconds" val NumQuotaSamplesProp = "quota.window.num" val QuotaWindowSizeSecondsProp = "quota.window.size.seconds" @@ -385,7 +385,7 @@ object KafkaConfig { val ConsumerQuotaOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default consumer quota. " + "Example: clientIdX=10485760,clientIdY=10485760" val QuotaDelayFactorDoc = "The quota delay factor modifies any delay by a fixed multiplier (default 1.2)" - val QuotaEnforcementDelaySecondsDoc = "The Sensor does not report quota violations for this amount of time after" + + val QuotaEnforcementBlackoutSecondsDoc = "The Sensor does not report quota violations for this amount of time after" + " the sensor is created. This is meant to prevent throttling all clients upon server startup" val NumQuotaSamplesDoc = "The number of samples to retain in memory" val QuotaWindowSizeSecondsDoc = "The time span of each sample" @@ -511,14 +511,14 @@ object KafkaConfig { .define(CompressionTypeProp, STRING, Defaults.CompressionType, HIGH, CompressionTypeDoc) /** ********* Quota configuration ***********/ - .define(ProducerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ProducerQuotaDefaultBytesPerSecond, HIGH, ProducerQuotaDefaultBytesPerSecondDoc) - .define(ConsumerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ConsumerQuotaDefaultBytesPerSecond, HIGH, ConsumerQuotaDefaultBytesPerSecondDoc) + .define(ProducerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ProducerQuotaDefaultBytesPerSecond, atLeast(1), HIGH, ProducerQuotaDefaultBytesPerSecondDoc) + .define(ConsumerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ConsumerQuotaDefaultBytesPerSecond, atLeast(1), HIGH, ConsumerQuotaDefaultBytesPerSecondDoc) .define(ProducerQuotaOverridesProp, STRING, Defaults.ProducerQuotaOverrides, HIGH, ProducerQuotaOverridesDoc) .define(ConsumerQuotaOverridesProp, STRING, Defaults.ConsumerQuotaOverrides, HIGH, ConsumerQuotaOverridesDoc) - .define(QuotaDelayFactorProp, DOUBLE, Defaults.QuotaDelayFactor, LOW, QuotaDelayFactorDoc) - .define(QuotaEnforcementDelaySecondsProp, INT, Defaults.QuotaEnforcementDelaySeconds, LOW, QuotaEnforcementDelaySecondsProp) - .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, LOW, NumQuotaSamplesDoc) - .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, LOW, QuotaWindowSizeSecondsDoc) + .define(QuotaDelayFactorProp, DOUBLE, Defaults.QuotaDelayFactor, atLeast(1), LOW, QuotaDelayFactorDoc) + .define(QuotaEnforcementBlackoutSecondsProp, INT, Defaults.QuotaEnforcementBlackoutSeconds, atLeast(0), LOW, QuotaEnforcementBlackoutSecondsProp) + .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) + .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) } def configNames() = { @@ -642,6 +642,10 @@ object KafkaConfig { consumerQuotaDefaultBytesPerSecond = parsed.get(ConsumerQuotaDefaultBytesPerSecondProp).asInstanceOf[Long], producerQuotaOverrides = parsed.get(ProducerQuotaOverridesProp).asInstanceOf[String], consumerQuotaOverrides = parsed.get(ConsumerQuotaOverridesProp).asInstanceOf[String], + quotaDelayFactor = parsed.get(QuotaDelayFactorProp).asInstanceOf[Double], + quotaEnforcementBlackoutSeconds = parsed.get(QuotaEnforcementBlackoutSecondsProp).asInstanceOf[Int], + numQuotaSamples = parsed.get(NumQuotaSamplesProp).asInstanceOf[Int], + quotaWindowSizeSeconds = parsed.get(QuotaWindowSizeSecondsProp).asInstanceOf[Int], deleteTopicEnable = parsed.get(DeleteTopicEnableProp).asInstanceOf[Boolean], compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String] ) @@ -791,6 +795,10 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ val consumerQuotaDefaultBytesPerSecond: Long = Defaults.ConsumerQuotaDefaultBytesPerSecond, val producerQuotaOverrides : String = Defaults.ProducerQuotaOverrides, val consumerQuotaOverrides : String = Defaults.ConsumerQuotaOverrides, + val quotaDelayFactor : Double = Defaults.QuotaDelayFactor, + val quotaEnforcementBlackoutSeconds : Int = Defaults.QuotaEnforcementBlackoutSeconds, + val numQuotaSamples : Int = Defaults.NumQuotaSamples, + val quotaWindowSizeSeconds : Int = Defaults.QuotaWindowSizeSeconds, val deleteTopicEnable: Boolean = Defaults.DeleteTopicEnable, val compressionType: String = Defaults.CompressionType ) { @@ -1016,6 +1024,10 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ props.put(ConsumerQuotaDefaultBytesPerSecondProp, consumerQuotaDefaultBytesPerSecond.toString) props.put(ProducerQuotaOverridesProp, producerQuotaOverrides.toString) props.put(ConsumerQuotaOverridesProp, consumerQuotaOverrides.toString) + props.put(QuotaDelayFactorProp, quotaDelayFactor.toString) + props.put(QuotaEnforcementBlackoutSecondsProp, quotaEnforcementBlackoutSeconds.toString) + props.put(NumQuotaSamplesProp, numQuotaSamples.toString) + props.put(QuotaWindowSizeSecondsProp, quotaWindowSizeSeconds.toString) props.put(DeleteTopicEnableProp, deleteTopicEnable.toString) props.put(CompressionTypeProp, compressionType.toString) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 5d5aba2..9f32b79 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -149,7 +149,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /* start processing requests */ apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, - kafkaController, zkClient, config.brokerId, config, metadataCache) + kafkaController, zkClient, config.brokerId, config, metadataCache, metrics) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 6b6281e..270e303 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -29,6 +29,7 @@ import kafka.message.{ByteBufferMessageSet, MessageSet} import java.util.concurrent.atomic.AtomicBoolean import java.io.{IOException, File} import java.util.concurrent.TimeUnit +import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import scala.Predef._ @@ -93,7 +94,7 @@ class ReplicaManager(val config: KafkaConfig, val zkClient: ZkClient, scheduler: Scheduler, val logManager: LogManager, - val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup { + val isShuttingDown: AtomicBoolean) extends Logging with KafkaMetricsGroup { /* epoch of the controller that last changed the leader */ @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1 private val localBrokerId = config.brokerId @@ -111,14 +112,6 @@ class ReplicaManager(val config: KafkaConfig, val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch]( purgatoryName = "Fetch", config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) - // Method 1 - val clientMetrics = new ClientQuotaMetrics(null, null) - - // Method 2 - val producerMetrics = new ClientQuotaMetrics2(null, null, "producer") - val consumerMetrics = new ClientQuotaMetrics2(null, null, "consumer") - - newGauge( "LeaderCount", new Gauge[Int] { @@ -288,12 +281,6 @@ class ReplicaManager(val config: KafkaConfig, messagesPerPartition: Map[TopicAndPartition, MessageSet], responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { - // Example of method 1 - clientMetrics.recordBytesProduced("xx", 1) - - // Method 2 - producerMetrics.record("xx", 1) - if (isValidRequiredAcks(requiredAcks)) { val sTime = SystemTime.milliseconds @@ -432,13 +419,6 @@ class ReplicaManager(val config: KafkaConfig, fetchMinBytes: Int, fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { - - // Example of method 1 - clientMetrics.recordBytesConsumed("xx", 1) - - // Method 2 - consumerMetrics.record("xx", 1) - val isFromFollower = replicaId >= 0 val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) diff --git a/core/src/main/scala/kafka/server/ThrottledRequest.scala b/core/src/main/scala/kafka/server/ThrottledRequest.scala new file mode 100644 index 0000000..2f62fef --- /dev/null +++ b/core/src/main/scala/kafka/server/ThrottledRequest.scala @@ -0,0 +1,22 @@ +package kafka.server + +import java.util.concurrent.{TimeUnit, Delayed} + +import kafka.utils.Time + +private[server] class ThrottledRequest(val time: Time, val delayTimeMs : Long, callback: => Unit) extends Delayed { + val endTime = time.milliseconds + delayTimeMs + + def execute() = callback + + def getDelay(unit: TimeUnit): Long = { + unit.convert(endTime - time.milliseconds, TimeUnit.MILLISECONDS) + } + + def compareTo(d: Delayed): Int = { + val other = d.asInstanceOf[ThrottledRequest] + if(this.endTime < other.endTime) -1 + else if(this.endTime > other.endTime) 1 + else 0 + } +} diff --git a/core/src/main/scala/kafka/utils/ShutdownableThread.scala b/core/src/main/scala/kafka/utils/ShutdownableThread.scala index fc226c8..db21aae 100644 --- a/core/src/main/scala/kafka/utils/ShutdownableThread.scala +++ b/core/src/main/scala/kafka/utils/ShutdownableThread.scala @@ -51,6 +51,9 @@ abstract class ShutdownableThread(val name: String, val isInterruptible: Boolean info("Shutdown completed") } + /** + * This method is continually in a loop called until the thread shuts down or this method throws an exception + */ def doWork(): Unit override def run(): Unit = { diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala new file mode 100644 index 0000000..70245f6 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -0,0 +1,144 @@ +/** + * Copyright 2015 Confluent Inc. + * + * 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.api + +import java.util.Properties + +import junit.framework.Assert +import kafka.consumer.SimpleConsumer +import kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.{ShutdownableThread, TestUtils} +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.producer._ +import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.{Sensor, KafkaMetric, Metrics} +import org.junit.Assert._ +import org.junit.Test + +import scala.collection.mutable + +class QuotasTest extends KafkaServerTestHarness { + private val producerBufferSize = 30000 + private val producerId1 = "QuotasTestProducer-1" + private val producerId2 = "QuotasTestProducer-2" + private val serverMessageMaxBytes = producerBufferSize / 2 + val numServers = 1 + val overridingProps = new Properties() + + // Low enough quota that a producer sending a small payload in a tight loop should get throttled + overridingProps.put(KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp, "50000") + overridingProps.put(KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp, "25000") + // unthrottled + overridingProps.put(KafkaConfig.ProducerQuotaOverridesProp, producerId2 + "=" + Long.MaxValue) + overridingProps.put(KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp, "25000") + overridingProps.put(KafkaConfig.QuotaEnforcementBlackoutSecondsProp, "1") + + override def generateConfigs() = { + FixedPortTestUtils.createBrokerConfigs(numServers, + zkConnect, + enableControlledShutdown = false) + .map(KafkaConfig.fromProps(_, overridingProps)) + } + + var producers = mutable.Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + var consumers = mutable.Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + + private val topic1 = "topic-1" + + override def setUp() { + super.setUp() + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.ACKS_CONFIG, "0") + producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "false") + producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, producerBufferSize.toString) + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, producerId1) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producers += new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, producerId2) + producers += new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + + // Create consumers + val consumerProps = new Properties + consumerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "QuotasTest") + consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + for(i <- 0 until 2) + consumers += new KafkaConsumer(consumerProps) + + val numPartitions = 1 + val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) + assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined)) + } + + override def tearDown() { + producers.foreach( _.close ) + consumers.foreach( _.close ) + super.tearDown() + } + + @Test + def testThrottledProducer() { + val allMetrics: mutable.Map[MetricName, KafkaMetric] = produce(producers(0), 1000) + + var numAsserts = 0 + for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { + if (name.tags().containsValue(producerId1) && name.name().startsWith("throttle-time")) { + Assert.assertTrue("Should have been throttled", metric.value() > 0) + numAsserts += 1 + } + } + // Should have matched 2 metrics + Assert.assertEquals(2, numAsserts) + } + + @Test + def testProducerOverrideUnthrottled() { + val allMetrics: mutable.Map[MetricName, KafkaMetric] = produce(producers(1), 1000) + + var numAsserts = 0 + for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { + if (name.tags().containsValue(producerId2) && name.name().startsWith("throttle-time")) { + Assert.assertFalse("Should not have been throttled", metric.value() > 0) + numAsserts += 1 + } + } + // Should have matched 2 metrics + Assert.assertEquals(2, numAsserts) + } + + def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): mutable.Map[MetricName, KafkaMetric] = { + val responses = + for (i <- 0 to count) { + p.send(new ProducerRecord[Array[Byte], Array[Byte]](topic1, null, null, i.toString.getBytes), + new ErrorLoggingCallback(topic1, null, null, true)).get() + Thread.sleep(1) + } + import scala.collection.JavaConverters._ + servers.head.metrics.metrics().asScala + } +} diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala index bcd86c1..3eb1945 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala @@ -18,7 +18,7 @@ package kafka.server import java.util.Collections -import org.apache.kafka.common.metrics.{Quota, MetricConfig} +import org.apache.kafka.common.metrics.{QuotaViolationException, Quota, MetricConfig} import org.apache.kafka.common.utils.MockTime import org.scalatest.junit.JUnit3Suite import org.junit.{Test, Assert} @@ -27,75 +27,50 @@ class ClientQuotaMetricsTest extends JUnit3Suite { private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), Collections.emptyList(), new MockTime) - private val config = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, - defaultConsumerQuotaBytesPerSecond = 1000, - producerQuotaOverrides = "p1=2000,p2=4000", - consumerQuotaOverrides = "c1=2000,c2=4000") + private val config = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, + quotaOverrides = "p1=2000,p2=4000") @Test def testQuotaParsing() { - val clientMetrics = new ClientQuotaMetrics(config, metrics) + val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer") Assert.assertEquals("Default producer quota should be 500", - new Quota(500, true), clientMetrics.producerQuota("random-client-id")) - Assert.assertEquals("Default consumer quota should be 1000", - new Quota(1000, true), clientMetrics.consumerQuota("random-client-id")) + new Quota(500, true), clientMetrics.quota("random-client-id")) Assert.assertEquals("Should return the overridden value (2000)", - new Quota(2000, true), clientMetrics.producerQuota("p1")) + new Quota(2000, true), clientMetrics.quota("p1")) Assert.assertEquals("Should return the overridden value (4000)", - new Quota(4000, true), clientMetrics.producerQuota("p2")) - Assert.assertEquals("Should return the overridden value (2000)", - new Quota(2000, true), clientMetrics.consumerQuota("c1")) - Assert.assertEquals("Should return the overridden value (4000)", - new Quota(4000, true), clientMetrics.consumerQuota("c2")) + new Quota(4000, true), clientMetrics.quota("p2")) } @Test - def testProducerQuotaViolation() { + def testQuotaViolation() { val time = new MockTime val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), Collections.emptyList(), time) - val clientMetrics = new ClientQuotaMetrics(config, metrics) + val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer") /* We have 10 second windows. Make sure that there is no quota violation * if we produce under the quota */ for(i <- 0 until 10) { - Assert.assertEquals(0, clientMetrics.recordBytesProduced("unknown", 400)) + clientMetrics.record("unknown", 400) time.sleep(1000) } // Create a spike. - val sleepTime = clientMetrics.recordBytesProduced("unknown", 2000) + var sleepTime = 0 + try { + clientMetrics.record("unknown", 2000) + } + catch { + case qve : QuotaViolationException => + sleepTime = qve.getDelayTimeMs + } // 400*10 + 2000 = 6000/10 = 600 bytes per second. // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200*1.2 delayfactor = 2640 Assert.assertEquals("Should be throttled", 2640, sleepTime) time.sleep(sleepTime) // At the end of sleep, the - Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.recordBytesProduced("unknown", 0)) - } - - @Test - def testConsumerQuotaViolation() { - - val time = new MockTime - val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), - Collections.emptyList(), - time) - - val clientMetrics = new ClientQuotaMetrics(config, metrics) - for(i <- 0 until 10) { - Assert.assertEquals(0, clientMetrics.recordBytesConsumed("c1", 1600)) - time.sleep(1000) - } - - // Create a spike. - val sleepTime = clientMetrics.recordBytesConsumed("c1", 8000) - // 1600*10 + 8000 = 24000/10 = 2400 bytes per second. - // (2400 - quota)/quota*window-size = (2400-2000)/2000*11 seconds = 2200*1.2 delayfactor = 2640 - Assert.assertEquals("Should be throttled", 2640, sleepTime) - time.sleep(sleepTime) - // At the end of sleep, the - Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.recordBytesConsumed("c1", 0)) + //Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.record("unknown", 0)) } @Test @@ -106,58 +81,36 @@ class ClientQuotaMetricsTest extends JUnit3Suite { time) // Case 1 - Default config var testConfig = ClientQuotaMetricsConfig() - var clientMetrics = new ClientQuotaMetrics(testConfig, metrics) - Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultProducerQuotaBytesPerSecond, true), - clientMetrics.producerQuota("p1")) - Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultConsumerQuotaBytesPerSecond, true), - clientMetrics.consumerQuota("p1")) + var clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer") + Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultQuotaPerSecond, true), + clientMetrics.quota("p1")) // Case 2 - Empty override - testConfig = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, - defaultConsumerQuotaBytesPerSecond = 1000, - producerQuotaOverrides = "p1=2000,p2=4000,,", - consumerQuotaOverrides = "c1=2000,c2=4000") + testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, + quotaOverrides = "p1=2000,p2=4000,,") - clientMetrics = new ClientQuotaMetrics(testConfig, metrics) - Assert.assertEquals(new Quota(2000, true), clientMetrics.producerQuota("p1")) - Assert.assertEquals(new Quota(4000, true), clientMetrics.producerQuota("p2")) - - // Case 3 - NumberFormatException for producer override - testConfig = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, - defaultConsumerQuotaBytesPerSecond = 1000, - producerQuotaOverrides = "p1=2000,p2=4000,p3=p4", - consumerQuotaOverrides = "c1=2000,c2=4000") - try { - clientMetrics = new ClientQuotaMetrics(testConfig, metrics) - Assert.fail("Should fail to parse invalid config " + testConfig.producerQuotaOverrides) - } - catch { - // Swallow. - case nfe : NumberFormatException => - } + clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer") + Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) - // Case 4 - NumberFormatException for consumer override - testConfig = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, - defaultConsumerQuotaBytesPerSecond = 1000, - producerQuotaOverrides = "p1=2000,p2=4000", - consumerQuotaOverrides = "c1=2000,c2=4000,c3=c4") + // Case 3 - NumberFormatException for override + testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, + quotaOverrides = "p1=2000,p2=4000,p3=p4") try { - clientMetrics = new ClientQuotaMetrics(testConfig, metrics) - Assert.fail("Should fail to parse invalid config " + testConfig.consumerQuotaOverrides) + clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer") + Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) } catch { // Swallow. case nfe : NumberFormatException => } - // Case 5 - IllegalArgumentException for producer override - testConfig = ClientQuotaMetricsConfig(defaultProducerQuotaBytesPerSecond = 500, - defaultConsumerQuotaBytesPerSecond = 1000, - producerQuotaOverrides = "p1=2000=3000", - consumerQuotaOverrides = "c1=2000,c2=4000") + // Case 4 - IllegalArgumentException for override + testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, + quotaOverrides = "p1=2000=3000") try { - clientMetrics = new ClientQuotaMetrics(testConfig, metrics) - Assert.fail("Should fail to parse invalid config " + testConfig.producerQuotaOverrides) + clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "producer") + Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) } catch { // Swallow. diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest2.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest2.scala deleted file mode 100644 index 5f3c5e1..0000000 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest2.scala +++ /dev/null @@ -1,121 +0,0 @@ -/** - * 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.Collections - -import org.apache.kafka.common.metrics.{QuotaViolationException, Quota, MetricConfig} -import org.apache.kafka.common.utils.MockTime -import org.scalatest.junit.JUnit3Suite -import org.junit.{Test, Assert} - -class ClientQuotaMetricsTest2 extends JUnit3Suite { - private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), - Collections.emptyList(), - new MockTime) - private val config = ClientQuotaMetricsConfig2(defaultQuotaPerSecond = 500, - quotaOverrides = "p1=2000,p2=4000") - - @Test - def testQuotaParsing() { - val clientMetrics = new ClientQuotaMetrics2(config, metrics, "producer") - Assert.assertEquals("Default producer quota should be 500", - new Quota(500, true), clientMetrics.quota("random-client-id")) - Assert.assertEquals("Should return the overridden value (2000)", - new Quota(2000, true), clientMetrics.quota("p1")) - Assert.assertEquals("Should return the overridden value (4000)", - new Quota(4000, true), clientMetrics.quota("p2")) - } - - @Test - def testQuotaViolation() { - val time = new MockTime - val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), - Collections.emptyList(), - time) - val clientMetrics = new ClientQuotaMetrics2(config, metrics, "producer") - /* We have 10 second windows. Make sure that there is no quota violation - * if we produce under the quota - */ - for(i <- 0 until 10) { - clientMetrics.record("unknown", 400) - time.sleep(1000) - } - - // Create a spike. - var sleepTime = 0 - try { - clientMetrics.record("unknown", 2000) - } - catch { - case qve : QuotaViolationException => - sleepTime = qve.getDelayTimeMs - } - // 400*10 + 2000 = 6000/10 = 600 bytes per second. - // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200*1.2 delayfactor = 2640 - Assert.assertEquals("Should be throttled", 2640, sleepTime) - time.sleep(sleepTime) - // At the end of sleep, the - //Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.record("unknown", 0)) - } - - @Test - def testOverrideParse() { - val time = new MockTime - val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), - Collections.emptyList(), - time) - // Case 1 - Default config - var testConfig = ClientQuotaMetricsConfig2() - var clientMetrics = new ClientQuotaMetrics2(testConfig, metrics, "consumer") - Assert.assertEquals(new Quota(ClientQuotaMetricsConfig2.DefaultQuotaPerSecond, true), - clientMetrics.quota("p1")) - - // Case 2 - Empty override - testConfig = ClientQuotaMetricsConfig2(defaultQuotaPerSecond = 500, - quotaOverrides = "p1=2000,p2=4000,,") - - clientMetrics = new ClientQuotaMetrics2(testConfig, metrics, "consumer") - Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) - Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) - - // Case 3 - NumberFormatException for override - testConfig = ClientQuotaMetricsConfig2(defaultQuotaPerSecond = 500, - quotaOverrides = "p1=2000,p2=4000,p3=p4") - try { - clientMetrics = new ClientQuotaMetrics2(testConfig, metrics, "consumer") - Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) - } - catch { - // Swallow. - case nfe : NumberFormatException => - } - - // Case 4 - IllegalArgumentException for override - testConfig = ClientQuotaMetricsConfig2(defaultQuotaPerSecond = 500, - quotaOverrides = "p1=2000=3000") - try { - clientMetrics = new ClientQuotaMetrics2(testConfig, metrics, "producer") - Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) - } - catch { - // Swallow. - case nfe : IllegalArgumentException => - } - - } -} diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 8014a5a..3a82844 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -218,6 +218,14 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.OffsetsRetentionMinutesProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.OffsetCommitTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.ProducerQuotaOverridesProp => expected.setProperty(name, "P1=10,P2=20") + case KafkaConfig.ConsumerQuotaOverridesProp => expected.setProperty(name, "C1=10,C2=20") + case KafkaConfig.QuotaDelayFactorProp => expected.setProperty(name, "%.1f".format(nextDouble + 1)) + case KafkaConfig.QuotaEnforcementBlackoutSecondsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.NumQuotaSamplesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.QuotaWindowSizeSecondsProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.DeleteTopicEnableProp => expected.setProperty(name, randFrom("true", "false")) // explicit, non trivial validations or with transient dependencies @@ -340,7 +348,14 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.OffsetsRetentionCheckIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.OffsetCommitTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") - + case KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ProducerQuotaOverridesProp => // ignore string + case KafkaConfig.ConsumerQuotaOverridesProp => // ignore string + case KafkaConfig.QuotaDelayFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.QuotaEnforcementBlackoutSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") + case KafkaConfig.NumQuotaSamplesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.QuotaWindowSizeSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") diff --git a/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala new file mode 100644 index 0000000..89509cf --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala @@ -0,0 +1,70 @@ +package kafka.server + + +import java.util.concurrent.{TimeUnit, DelayQueue} + +import kafka.utils.TestUtils +import org.junit.{Before, Assert, Test} +import org.scalatest.junit.JUnit3Suite + +class ThrottledRequestExpirationTest extends JUnit3Suite { + val time = new kafka.utils.MockTime + var numCallbacks : Int = 0 + + def callback { + numCallbacks += 1 + } + + @Before + def beforeMethod(): Unit = { + numCallbacks = 0 + } + + @Test + def testExpire() { + val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) + val config = KafkaConfig.fromProps(props) + val api = new KafkaApis(null, null, null, null, null, null, 1, config, null, null) + val delayQueue = new DelayQueue[ThrottledRequest]() + val reaper = new api.ThrottledRequestReaper(delayQueue) + + var numCallbacks : Int = 0 + def callback { + numCallbacks += 1 + } + // Add 4 elements to the queue out of order. Add 2 elements with the same expire timestamp + delayQueue.add(new ThrottledRequest(time, 10, callback)) + delayQueue.add(new ThrottledRequest(time, 30, callback)) + delayQueue.add(new ThrottledRequest(time, 30, callback)) + delayQueue.add(new ThrottledRequest(time, 20, callback)) + + for(itr <- 1 to 3) { + time.sleep(10) + reaper.doWork() + Assert.assertEquals(itr, numCallbacks) + + } + reaper.doWork() + Assert.assertEquals(4, numCallbacks) + Assert.assertEquals(0, delayQueue.size()) + reaper.doWork() + Assert.assertEquals(4, numCallbacks) + } + + @Test + def testThrottledRequest() { + val t1 : ThrottledRequest = new ThrottledRequest(time, 10, callback) + val t2 : ThrottledRequest = new ThrottledRequest(time, 20, callback) + val t3 : ThrottledRequest = new ThrottledRequest(time, 20, callback) + Assert.assertEquals(10, t1.delayTimeMs) + Assert.assertEquals(20, t2.delayTimeMs) + Assert.assertEquals(20, t3.delayTimeMs) + + for(itr <- 0 to 2) { + Assert.assertEquals(10 - 10*itr, t1.getDelay(TimeUnit.MILLISECONDS)) + Assert.assertEquals(20 - 10*itr, t2.getDelay(TimeUnit.MILLISECONDS)) + Assert.assertEquals(20 - 10*itr, t3.getDelay(TimeUnit.MILLISECONDS)) + time.sleep(10) + } + } +} -- 1.7.12.4 From 1abfb188a3d63c3dbb6e47bfbb2365d20175f511 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 4 May 2015 19:44:49 -0700 Subject: [PATCH 07/10] Incorporated Jun's comments --- core/src/main/scala/kafka/server/ClientQuotaMetrics.scala | 9 ++------- core/src/test/scala/integration/kafka/api/QuotasTest.scala | 8 ++++---- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index e2c0e74..ccae868 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -153,16 +153,11 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, if(quotaSensor == null) { // create the throttle time sensor also throttleTimeSensor = metrics.sensor(throttleTimeSensorName) - throttleTimeSensor.add(new MetricName("throttle-time-max", + throttleTimeSensor.add(new MetricName("throttle-time", apiKey, "Tracking throttle-time per client", "client-id", - clientId), new Max()) - throttleTimeSensor.add(new MetricName("throttle-time-avg", - apiKey, - "Tracking throttle-time per client", - "client-id", - clientId), new Avg()) + clientId), new Rate()) quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota(clientId))) quotaSensor.add(new MetricName("byte-rate", apiKey, diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index 70245f6..a604dd8 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -112,8 +112,8 @@ class QuotasTest extends KafkaServerTestHarness { numAsserts += 1 } } - // Should have matched 2 metrics - Assert.assertEquals(2, numAsserts) + // Should have matched 1 metric + Assert.assertEquals(1, numAsserts) } @Test @@ -127,8 +127,8 @@ class QuotasTest extends KafkaServerTestHarness { numAsserts += 1 } } - // Should have matched 2 metrics - Assert.assertEquals(2, numAsserts) + // Should have matched 1 metric + Assert.assertEquals(1, numAsserts) } def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): mutable.Map[MetricName, KafkaMetric] = { -- 1.7.12.4 From 3b03604d187a471bd55d641d736f69ac16cbfb6c Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 5 May 2015 10:37:11 -0700 Subject: [PATCH 08/10] Adding javadoc --- core/src/main/scala/kafka/server/KafkaApis.scala | 6 +++--- .../main/scala/kafka/server/ThrottledRequest.scala | 23 ++++++++++++++++++++++ .../server/ThrottledRequestExpirationTest.scala | 20 +++++++++++++++++-- 3 files changed, 44 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6bf4b04..46c2be6 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -67,7 +67,6 @@ class KafkaApis(val requestChannel: RequestChannel, val response : ThrottledRequest = delayQueue.poll(1, TimeUnit.SECONDS) if(response != null) { trace("Response throttled for: " + response.delayTimeMs + " ms") - //System.out.println("Response throttled for: " + response.delayTimeMs + " ms") response.execute() } } @@ -262,7 +261,6 @@ class KafkaApis(val requestChannel: RequestChannel, catch { case qve : QuotaViolationException => throttleTime = qve.getDelayTimeMs - //System.out.println("Request throttled for " + throttleTime + " ms") } case None => warn("Cannot throttle Api key " + key) @@ -351,7 +349,6 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] - System.out.println("Fetching " + fetchRequest.clientId) // the callback for sending a fetch response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { @@ -654,6 +651,9 @@ class KafkaApis(val requestChannel: RequestChannel, sendResponseCallback) } + /* + * Returns a Map of all quota managers configured. The request Api key is the key for the Map + */ private def instantiateQuotaManagers(cfg : KafkaConfig): Map[String, ClientQuotaMetrics] = { val producerQuotaManagerCfg = ClientQuotaMetricsConfig( defaultQuotaPerSecond = cfg.producerQuotaDefaultBytesPerSecond, diff --git a/core/src/main/scala/kafka/server/ThrottledRequest.scala b/core/src/main/scala/kafka/server/ThrottledRequest.scala index 2f62fef..d825df3 100644 --- a/core/src/main/scala/kafka/server/ThrottledRequest.scala +++ b/core/src/main/scala/kafka/server/ThrottledRequest.scala @@ -1,9 +1,32 @@ +/** + * 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, Delayed} import kafka.utils.Time +/** + * Represents a request whose response has been delayed. + * @param time @Time instance to use + * @param delayTimeMs delay associated with this request + * @param callback Callback to trigger after delayTimeMs milliseconds + */ private[server] class ThrottledRequest(val time: Time, val delayTimeMs : Long, callback: => Unit) extends Delayed { val endTime = time.milliseconds + delayTimeMs diff --git a/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala index 89509cf..f4502e3 100644 --- a/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala @@ -1,5 +1,21 @@ -package kafka.server +/** + * 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, DelayQueue} @@ -16,7 +32,7 @@ class ThrottledRequestExpirationTest extends JUnit3Suite { } @Before - def beforeMethod(): Unit = { + def beforeMethod() { numCallbacks = 0 } -- 1.7.12.4 From f8dcfd7d4f5a0d6a361759cf13be25a7c6bbb34f Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 5 May 2015 15:26:27 -0700 Subject: [PATCH 09/10] KAFKA-2084 - Moved the callbacks to ClientQuotaMetrics --- .../scala/kafka/server/ClientQuotaMetrics.scala | 48 +++++++++++--- core/src/main/scala/kafka/server/KafkaApis.scala | 76 ++++++---------------- .../main/scala/kafka/server/ThrottledRequest.scala | 3 +- .../unit/kafka/server/ClientQuotaMetricsTest.scala | 62 ++++++++++-------- .../server/ThrottledRequestExpirationTest.scala | 22 +++---- 5 files changed, 108 insertions(+), 103 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index ccae868..d050c55 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -16,14 +16,16 @@ */ package kafka.server -import java.util.concurrent.TimeUnit +import java.util.concurrent.{DelayQueue, TimeUnit} -import kafka.utils.Logging +import kafka.utils.{ShutdownableThread, Logging} import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ -import org.apache.kafka.common.metrics.stats.{Avg, Max, Rate} +import org.apache.kafka.common.metrics.stats.Rate import java.util.concurrent.locks.ReentrantReadWriteLock +import org.apache.kafka.common.utils.Time + import scala.collection.mutable /** @@ -69,32 +71,57 @@ object ClientQuotaMetricsConfig { * @param metrics @Metrics Metrics instance */ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, - private val metrics : Metrics, - private val apiKey : String) extends Logging { + private val metrics : Metrics, + private val apiKey : String, + private val time : Time) extends Logging { private val overriddenQuota = initQuotaMap(config.quotaOverrides) private val defaultQuota = Quota.lessThan(config.defaultQuotaPerSecond) private val lock = new ReentrantReadWriteLock() + private val delayQueue = new DelayQueue[ThrottledRequest]() + val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) + throttledRequestReaper.start() + + /** + * Reaper thread that triggers callbacks on all throttled requests + * @param delayQueue DelayQueue to dequeue from + */ + class ThrottledRequestReaper(delayQueue: DelayQueue[ThrottledRequest]) extends ShutdownableThread( + "ThrottledRequestReaper-%s".format(apiKey), false) { + + override def doWork(): Unit = { + val response : ThrottledRequest = delayQueue.poll(1, TimeUnit.SECONDS) + if(response != null) { + trace("Response throttled for: " + response.delayTimeMs + " ms") + response.execute() + } + } + } /** * Records that a produce request wrote some data * @param clientId clientId that produced the data * @param value amount of data written in bytes + * @param callback Callback function. This will be triggered immediately if quota is not violated. + * If there is a quota violation, this callback will be triggered after a delay * @return Number of milliseconds to delay the response in case of Quota violation. * Zero otherwise */ - def record(clientId: String, value : Int) = { + def record(clientId: String, value : Int, callback: => Unit) : Int = { val sensors = getOrCreateQuotaSensors(clientId) var delayTime = 0.0 try { sensors._1.record(value) + // trigger the callback immediately if quota is not violated + callback } catch { case qve : QuotaViolationException => delayTime = qve.getDelayTimeMs*config.quotaDelayFactor - qve.setDelayTimeMs(delayTime.toInt) + delayQueue.add(new ThrottledRequest(time, delayTime.toLong, callback)) sensors._2.record(delayTime) + // If delayed, add the element to the delayQueue logger.warn("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), qve) - throw qve } + delayTime.toInt } /** @@ -195,4 +222,9 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, } output } + + def shutdown() = { + throttledRequestReaper.shutdown() + throttledRequestReaper.awaitShutdown() + } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 46c2be6..bf8d08c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -18,10 +18,7 @@ package kafka.server -import java.util.concurrent.{TimeUnit, DelayQueue} - -import kafka.utils._ -import org.apache.kafka.common.metrics.{QuotaViolationException, Metrics} +import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.requests.{JoinGroupResponse, JoinGroupRequest, HeartbeatRequest, HeartbeatResponse, ResponseHeader} import org.apache.kafka.common.TopicPartition @@ -56,21 +53,6 @@ class KafkaApis(val requestChannel: RequestChannel, this.logIdent = "[KafkaApi-%d] ".format(brokerId) // Store all the quota managers for each type of request private val quotaManagers = instantiateQuotaManagers(config) - private val delayQueue = new DelayQueue[ThrottledRequest]() - private val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) - throttledRequestReaper.start() - - class ThrottledRequestReaper(delayQueue: DelayQueue[ThrottledRequest]) extends ShutdownableThread( - "ThrottledRequestReaper-%d".format(brokerId), false) { - - override def doWork(): Unit = { - val response : ThrottledRequest = delayQueue.poll(1, TimeUnit.SECONDS) - if(response != null) { - trace("Response throttled for: " + response.delayTimeMs + " ms") - response.execute() - } - } - } /** * Top-level method that handles all requests and multiplexes to the right api @@ -250,29 +232,6 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def throttleOrRespond(clientId : String, key : String, value: Int, callback: => Unit) = { - var throttleTime = 0 - quotaManagers.get(key) match { - case Some(quotaManager) => - try { - quotaManager.record(clientId, value) - - } - catch { - case qve : QuotaViolationException => - throttleTime = qve.getDelayTimeMs - } - case None => - warn("Cannot throttle Api key " + key) - } - if(throttleTime > 0) { - delayQueue.add(new ThrottledRequest(SystemTime, throttleTime, callback)) - } - else { - callback - } - } - /** * Handle a produce request */ @@ -320,10 +279,13 @@ class KafkaApis(val requestChannel: RequestChannel, } } - throttleOrRespond(produceRequest.clientId, - RequestKeys.nameForKey(RequestKeys.ProduceKey), - numBytesAppended, - produceResponseCallback) + val requestKey = RequestKeys.nameForKey(RequestKeys.ProduceKey) + quotaManagers.get(requestKey) match { + case Some(quotaManager) => + quotaManager.record(produceRequest.clientId, numBytesAppended, produceResponseCallback) + case None => + warn("Cannot throttle Api key %s".format(requestKey)) + } } // only allow appending to internal topic partitions @@ -369,10 +331,14 @@ class KafkaApis(val requestChannel: RequestChannel, def fetchResponseCallback { requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) } - throttleOrRespond(fetchRequest.clientId, - RequestKeys.nameForKey(RequestKeys.FetchKey), - response.sizeInBytes, - fetchResponseCallback) + + val requestKey = RequestKeys.nameForKey(RequestKeys.ProduceKey) + quotaManagers.get(requestKey) match { + case Some(quotaManager) => + quotaManager.record(fetchRequest.clientId, response.sizeInBytes, fetchResponseCallback) + case None => + warn("Cannot throttle Api key %s".format(requestKey)) + } } // call the replica manager to fetch messages from the local replica @@ -673,18 +639,18 @@ class KafkaApis(val requestChannel: RequestChannel, quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds ) + // Using SystemTime from kafka.common because Metrics use the system time from clients val quotaManagers = Map[String, ClientQuotaMetrics]( RequestKeys.nameForKey(RequestKeys.ProduceKey) -> - new ClientQuotaMetrics(producerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.ProduceKey)), + new ClientQuotaMetrics(producerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.ProduceKey), new org.apache.kafka.common.utils.SystemTime), RequestKeys.nameForKey(RequestKeys.FetchKey) -> - new ClientQuotaMetrics(consumerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.FetchKey)) + new ClientQuotaMetrics(consumerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.FetchKey), new org.apache.kafka.common.utils.SystemTime) ) quotaManagers } def close() { - throttledRequestReaper.shutdown() - throttledRequestReaper.awaitShutdown() - debug("Shut down complete.") + quotaManagers.foreach(entry => entry._2.shutdown()) + info("Shut down complete.") } } diff --git a/core/src/main/scala/kafka/server/ThrottledRequest.scala b/core/src/main/scala/kafka/server/ThrottledRequest.scala index d825df3..dbbca36 100644 --- a/core/src/main/scala/kafka/server/ThrottledRequest.scala +++ b/core/src/main/scala/kafka/server/ThrottledRequest.scala @@ -19,7 +19,8 @@ package kafka.server import java.util.concurrent.{TimeUnit, Delayed} -import kafka.utils.Time +import org.apache.kafka.common.utils.Time + /** * Represents a request whose response has been delayed. diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala index 3eb1945..17b5973 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala @@ -18,21 +18,32 @@ package kafka.server import java.util.Collections -import org.apache.kafka.common.metrics.{QuotaViolationException, Quota, MetricConfig} +import org.apache.kafka.common.metrics.{Quota, MetricConfig} import org.apache.kafka.common.utils.MockTime import org.scalatest.junit.JUnit3Suite -import org.junit.{Test, Assert} +import org.junit.{Before, Test, Assert} class ClientQuotaMetricsTest extends JUnit3Suite { + private val time = new MockTime private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), Collections.emptyList(), - new MockTime) + time) private val config = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, quotaOverrides = "p1=2000,p2=4000") + var numCallbacks : Int = 0 + def callback { + numCallbacks += 1 + } + + @Before + def beforeMethod() { + numCallbacks = 0 + } + @Test def testQuotaParsing() { - val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer") + val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer", time) Assert.assertEquals("Default producer quota should be 500", new Quota(500, true), clientMetrics.quota("random-client-id")) Assert.assertEquals("Should return the overridden value (2000)", @@ -43,45 +54,40 @@ class ClientQuotaMetricsTest extends JUnit3Suite { @Test def testQuotaViolation() { - val time = new MockTime - val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), - Collections.emptyList(), - time) - val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer") + val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer", time) /* We have 10 second windows. Make sure that there is no quota violation * if we produce under the quota */ for(i <- 0 until 10) { - clientMetrics.record("unknown", 400) + clientMetrics.record("unknown", 400, callback) time.sleep(1000) } + Assert.assertEquals(10, numCallbacks) // Create a spike. - var sleepTime = 0 - try { - clientMetrics.record("unknown", 2000) - } - catch { - case qve : QuotaViolationException => - sleepTime = qve.getDelayTimeMs - } // 400*10 + 2000 = 6000/10 = 600 bytes per second. // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200*1.2 delayfactor = 2640 + val sleepTime = clientMetrics.record("unknown", 2000, callback) Assert.assertEquals("Should be throttled", 2640, sleepTime) + // After a request is delayed, the callback cannot be triggered immediately + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(10, numCallbacks) time.sleep(sleepTime) - // At the end of sleep, the - //Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.record("unknown", 0)) + + // Callback can only be triggered after the the delay time passes + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(11, numCallbacks) + + // At the end of sleep, the client should be unthrottled. Callback count should increase after calling the method + Assert.assertEquals("Should be unthrottled again", 0, clientMetrics.record("unknown", 0, callback)) + Assert.assertEquals(12, numCallbacks) } @Test def testOverrideParse() { - val time = new MockTime - val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), - Collections.emptyList(), - time) // Case 1 - Default config var testConfig = ClientQuotaMetricsConfig() - var clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer") + var clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultQuotaPerSecond, true), clientMetrics.quota("p1")) @@ -89,7 +95,7 @@ class ClientQuotaMetricsTest extends JUnit3Suite { testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, quotaOverrides = "p1=2000,p2=4000,,") - clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer") + clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) @@ -97,7 +103,7 @@ class ClientQuotaMetricsTest extends JUnit3Suite { testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, quotaOverrides = "p1=2000,p2=4000,p3=p4") try { - clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer") + clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) } catch { @@ -109,7 +115,7 @@ class ClientQuotaMetricsTest extends JUnit3Suite { testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, quotaOverrides = "p1=2000=3000") try { - clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "producer") + clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "producer", time) Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) } catch { diff --git a/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala index f4502e3..a8ff7f0 100644 --- a/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala @@ -17,15 +17,22 @@ package kafka.server + +import java.util.Collections import java.util.concurrent.{TimeUnit, DelayQueue} -import kafka.utils.TestUtils +import org.apache.kafka.common.metrics.MetricConfig +import org.apache.kafka.common.utils.MockTime import org.junit.{Before, Assert, Test} import org.scalatest.junit.JUnit3Suite class ThrottledRequestExpirationTest extends JUnit3Suite { - val time = new kafka.utils.MockTime - var numCallbacks : Int = 0 + private val time = new MockTime + private var numCallbacks : Int = 0 + private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), + Collections.emptyList(), + time) + private val clientMetrics = new ClientQuotaMetrics(ClientQuotaMetricsConfig(), metrics, "producer", time) def callback { numCallbacks += 1 @@ -38,16 +45,9 @@ class ThrottledRequestExpirationTest extends JUnit3Suite { @Test def testExpire() { - val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) - val config = KafkaConfig.fromProps(props) - val api = new KafkaApis(null, null, null, null, null, null, 1, config, null, null) val delayQueue = new DelayQueue[ThrottledRequest]() - val reaper = new api.ThrottledRequestReaper(delayQueue) + val reaper = new clientMetrics.ThrottledRequestReaper(delayQueue) - var numCallbacks : Int = 0 - def callback { - numCallbacks += 1 - } // Add 4 elements to the queue out of order. Add 2 elements with the same expire timestamp delayQueue.add(new ThrottledRequest(time, 10, callback)) delayQueue.add(new ThrottledRequest(time, 30, callback)) -- 1.7.12.4 From f36e73784bec6c3c6656ff87bd4a74ab43fc30cd Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 5 May 2015 17:51:13 -0700 Subject: [PATCH 10/10] Adding more configs --- .../scala/kafka/server/ClientQuotaMetrics.scala | 20 ++++---- core/src/main/scala/kafka/server/KafkaApis.scala | 8 +-- core/src/main/scala/kafka/server/KafkaConfig.scala | 59 +++++++++++++++------- core/src/main/scala/kafka/server/KafkaServer.scala | 13 +++-- .../scala/integration/kafka/api/QuotasTest.scala | 4 +- .../unit/kafka/server/ClientQuotaMetricsTest.scala | 22 ++++---- .../kafka/server/KafkaConfigConfigDefTest.scala | 12 +++-- 7 files changed, 83 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index d050c55..5161a1f 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -30,8 +30,8 @@ import scala.collection.mutable /** * Configuration settings for quota management - * @param defaultQuotaPerSecond The default quota allocated to any client - * @param quotaOverrides The comma separated overrides per client. "c1=X,c2=Y" + * @param defaultQuotaBytesPerSecond The default bytes per second quota allocated to any client + * @param quotaBytesPerSecondOverrides The comma separated overrides per client. "c1=X,c2=Y" * @param quotaDelayFactor The quota delay factor modifies any delay by a fixed multiplier (default 1.2) * @param quotaEnforcementBlackoutSeconds The Sensor does not report quota violations for this amount of time after * the sensor is created. This is meant to prevent throttling all clients @@ -40,10 +40,10 @@ import scala.collection.mutable * @param quotaWindowSizeSeconds The time span of each sample * */ -case class ClientQuotaMetricsConfig(defaultQuotaPerSecond : Long = - ClientQuotaMetricsConfig.DefaultQuotaPerSecond, - quotaOverrides : String = - ClientQuotaMetricsConfig.QuotaOverrides, +case class ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond : Long = + ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond, + quotaBytesPerSecondOverrides : String = + ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides, quotaDelayFactor: Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, quotaEnforcementBlackoutSeconds : Int = @@ -54,8 +54,8 @@ case class ClientQuotaMetricsConfig(defaultQuotaPerSecond : Long = ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) object ClientQuotaMetricsConfig { - val DefaultQuotaPerSecond = Long.MaxValue - val QuotaOverrides = "" + val DefaultQuotaBytesPerSecond = Long.MaxValue + val QuotaBytesPerSecondOverrides = "" val DefaultQuotaDelayFactor = 1.2 val DefaultQuotaEnforcementBlackoutSeconds = 5 // Always have 10 whole windows + 1 current window @@ -74,8 +74,8 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, private val metrics : Metrics, private val apiKey : String, private val time : Time) extends Logging { - private val overriddenQuota = initQuotaMap(config.quotaOverrides) - private val defaultQuota = Quota.lessThan(config.defaultQuotaPerSecond) + private val overriddenQuota = initQuotaMap(config.quotaBytesPerSecondOverrides) + private val defaultQuota = Quota.lessThan(config.defaultQuotaBytesPerSecond) private val lock = new ReentrantReadWriteLock() private val delayQueue = new DelayQueue[ThrottledRequest]() val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index bf8d08c..eaef98b 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -622,8 +622,8 @@ class KafkaApis(val requestChannel: RequestChannel, */ private def instantiateQuotaManagers(cfg : KafkaConfig): Map[String, ClientQuotaMetrics] = { val producerQuotaManagerCfg = ClientQuotaMetricsConfig( - defaultQuotaPerSecond = cfg.producerQuotaDefaultBytesPerSecond, - quotaOverrides = cfg.producerQuotaOverrides, + defaultQuotaBytesPerSecond = cfg.producerQuotaDefaultBytesPerSecond, + quotaBytesPerSecondOverrides = cfg.producerQuotaBytesPerSecondOverrides, quotaDelayFactor = cfg.quotaDelayFactor, quotaEnforcementBlackoutSeconds = cfg.quotaEnforcementBlackoutSeconds, numQuotaSamples = cfg.numQuotaSamples, @@ -631,8 +631,8 @@ class KafkaApis(val requestChannel: RequestChannel, ) val consumerQuotaManagerCfg = ClientQuotaMetricsConfig( - defaultQuotaPerSecond = cfg.consumerQuotaDefaultBytesPerSecond, - quotaOverrides = cfg.consumerQuotaOverrides, + defaultQuotaBytesPerSecond = cfg.consumerQuotaDefaultBytesPerSecond, + quotaBytesPerSecondOverrides = cfg.consumerQuotaBytesPerSecondOverrides, quotaDelayFactor = cfg.quotaDelayFactor, quotaEnforcementBlackoutSeconds = cfg.quotaEnforcementBlackoutSeconds, numQuotaSamples = cfg.numQuotaSamples, diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c7242ef..2ffb73f 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -124,15 +124,19 @@ object Defaults { val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks /** ********* Quota Configuration ***********/ - val ProducerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaPerSecond - val ConsumerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaPerSecond - val ProducerQuotaOverrides = ClientQuotaMetricsConfig.QuotaOverrides - val ConsumerQuotaOverrides = ClientQuotaMetricsConfig.QuotaOverrides + val ProducerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond + val ConsumerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond + val ProducerQuotaBytesPerSecondOverrides = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides + val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides val QuotaDelayFactor : Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor val QuotaEnforcementBlackoutSeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaEnforcementBlackoutSeconds val NumQuotaSamples : Int = ClientQuotaMetricsConfig.DefaultNumQuotaSamples val QuotaWindowSizeSeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds + /** ********* Kafka Metrics Configuration ***********/ + val MetricsNumSamples : Int = 2 + val MetricsWindowSizeSeconds : Int = 30 + val DeleteTopicEnable = false val CompressionType = "producer" @@ -242,12 +246,15 @@ object KafkaConfig { /** ********* Quota Configuration ***********/ val ProducerQuotaDefaultBytesPerSecondProp = "quota.producer.default" val ConsumerQuotaDefaultBytesPerSecondProp = "quota.consumer.default" - val ProducerQuotaOverridesProp = "quota.producer.overrides" - val ConsumerQuotaOverridesProp = "quota.consumer.overrides" + val ProducerQuotaBytesPerSecondOverridesProp = "quota.producer.bytes.per.second.overrides" + val ConsumerQuotaBytesPerSecondOverridesProp = "quota.consumer.bytes.per.second.overrides" val QuotaDelayFactorProp = "quota.delay.factor" val QuotaEnforcementBlackoutSecondsProp = "quota.blackout.enforcement.seconds" val NumQuotaSamplesProp = "quota.window.num" val QuotaWindowSizeSecondsProp = "quota.window.size.seconds" + /** ********* Kafka Metrics Configuration ***********/ + val MetricsNumSamplesProp = "metrics.num.samples" + val MetricsWindowSizeSecondsProp = "metrics.sample.window.seconds" val DeleteTopicEnableProp = "delete.topic.enable" val CompressionTypeProp = "compression.type" @@ -380,15 +387,19 @@ object KafkaConfig { /** ********* Quota Configuration ***********/ val ProducerQuotaDefaultBytesPerSecondDoc = "Any producer distinguished by clientId will get throttled if it produces more bytes than this value per-second" val ConsumerQuotaDefaultBytesPerSecondDoc = "Any consumer distinguished by clientId/consumer group will get throttled if it fetches more bytes than this value per-second" - val ProducerQuotaOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default producer quota. " + + val ProducerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default producer quota. " + "Example: clientIdX=10485760,clientIdY=10485760" - val ConsumerQuotaOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default consumer quota. " + + val ConsumerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default consumer quota. " + "Example: clientIdX=10485760,clientIdY=10485760" val QuotaDelayFactorDoc = "The quota delay factor modifies any delay by a fixed multiplier (default 1.2)" val QuotaEnforcementBlackoutSecondsDoc = "The Sensor does not report quota violations for this amount of time after" + " the sensor is created. This is meant to prevent throttling all clients upon server startup" val NumQuotaSamplesDoc = "The number of samples to retain in memory" val QuotaWindowSizeSecondsDoc = "The time span of each sample" + /** ********* Kafka Metrics Configuration ***********/ + val MetricsNumSamplesDoc = "Number of samples to maintained to compute metrics. Only available for KafkaMetrics and not codahale" + val MetricsWindowSizeSecondsDoc = "Size in seconds of each sample maintained to compute metrics. Only available for KafkaMetrics and not codahale" + val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + "('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + @@ -513,12 +524,16 @@ object KafkaConfig { /** ********* Quota configuration ***********/ .define(ProducerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ProducerQuotaDefaultBytesPerSecond, atLeast(1), HIGH, ProducerQuotaDefaultBytesPerSecondDoc) .define(ConsumerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ConsumerQuotaDefaultBytesPerSecond, atLeast(1), HIGH, ConsumerQuotaDefaultBytesPerSecondDoc) - .define(ProducerQuotaOverridesProp, STRING, Defaults.ProducerQuotaOverrides, HIGH, ProducerQuotaOverridesDoc) - .define(ConsumerQuotaOverridesProp, STRING, Defaults.ConsumerQuotaOverrides, HIGH, ConsumerQuotaOverridesDoc) + .define(ProducerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ProducerQuotaBytesPerSecondOverrides, HIGH, ProducerQuotaBytesPerSecondOverridesDoc) + .define(ConsumerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ConsumerQuotaBytesPerSecondOverrides, HIGH, ConsumerQuotaBytesPerSecondOverridesDoc) .define(QuotaDelayFactorProp, DOUBLE, Defaults.QuotaDelayFactor, atLeast(1), LOW, QuotaDelayFactorDoc) - .define(QuotaEnforcementBlackoutSecondsProp, INT, Defaults.QuotaEnforcementBlackoutSeconds, atLeast(0), LOW, QuotaEnforcementBlackoutSecondsProp) + .define(QuotaEnforcementBlackoutSecondsProp, INT, Defaults.QuotaEnforcementBlackoutSeconds, atLeast(0), LOW, QuotaEnforcementBlackoutSecondsDoc) .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) + + /** ********* Kafka Metrics Configuration ***********/ + .define(MetricsNumSamplesProp, INT, Defaults.MetricsNumSamples, atLeast(1), LOW, MetricsNumSamplesDoc) + .define(MetricsWindowSizeSecondsProp, INT, Defaults.MetricsWindowSizeSeconds, atLeast(1), LOW, MetricsWindowSizeSecondsDoc) } def configNames() = { @@ -640,12 +655,15 @@ object KafkaConfig { /** ********* Quota configuration ***********/ producerQuotaDefaultBytesPerSecond = parsed.get(ProducerQuotaDefaultBytesPerSecondProp).asInstanceOf[Long], consumerQuotaDefaultBytesPerSecond = parsed.get(ConsumerQuotaDefaultBytesPerSecondProp).asInstanceOf[Long], - producerQuotaOverrides = parsed.get(ProducerQuotaOverridesProp).asInstanceOf[String], - consumerQuotaOverrides = parsed.get(ConsumerQuotaOverridesProp).asInstanceOf[String], + producerQuotaBytesPerSecondOverrides = parsed.get(ProducerQuotaBytesPerSecondOverridesProp).asInstanceOf[String], + consumerQuotaBytesPerSecondOverrides = parsed.get(ConsumerQuotaBytesPerSecondOverridesProp).asInstanceOf[String], quotaDelayFactor = parsed.get(QuotaDelayFactorProp).asInstanceOf[Double], quotaEnforcementBlackoutSeconds = parsed.get(QuotaEnforcementBlackoutSecondsProp).asInstanceOf[Int], numQuotaSamples = parsed.get(NumQuotaSamplesProp).asInstanceOf[Int], quotaWindowSizeSeconds = parsed.get(QuotaWindowSizeSecondsProp).asInstanceOf[Int], + /** ********* Kafka Metrics Configuration ***********/ + metricsNumSamples = parsed.get(MetricsNumSamplesProp).asInstanceOf[Int], + metricsWindowSizeSeconds = parsed.get(MetricsWindowSizeSecondsProp).asInstanceOf[Int], deleteTopicEnable = parsed.get(DeleteTopicEnableProp).asInstanceOf[Boolean], compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String] ) @@ -793,15 +811,16 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ /** ********* Quota configuration ***********/ val producerQuotaDefaultBytesPerSecond: Long = Defaults.ProducerQuotaDefaultBytesPerSecond, val consumerQuotaDefaultBytesPerSecond: Long = Defaults.ConsumerQuotaDefaultBytesPerSecond, - val producerQuotaOverrides : String = Defaults.ProducerQuotaOverrides, - val consumerQuotaOverrides : String = Defaults.ConsumerQuotaOverrides, + val producerQuotaBytesPerSecondOverrides : String = Defaults.ProducerQuotaBytesPerSecondOverrides, + val consumerQuotaBytesPerSecondOverrides : String = Defaults.ConsumerQuotaBytesPerSecondOverrides, val quotaDelayFactor : Double = Defaults.QuotaDelayFactor, val quotaEnforcementBlackoutSeconds : Int = Defaults.QuotaEnforcementBlackoutSeconds, val numQuotaSamples : Int = Defaults.NumQuotaSamples, val quotaWindowSizeSeconds : Int = Defaults.QuotaWindowSizeSeconds, + val metricsNumSamples : Int = Defaults.MetricsNumSamples, + val metricsWindowSizeSeconds : Int = Defaults.MetricsWindowSizeSeconds, val deleteTopicEnable: Boolean = Defaults.DeleteTopicEnable, - val compressionType: String = Defaults.CompressionType - ) { + val compressionType: String = Defaults.CompressionType) { val zkConnectionTimeoutMs: Int = _zkConnectionTimeoutMs.getOrElse(zkSessionTimeoutMs) @@ -1022,12 +1041,14 @@ class KafkaConfig(/** ********* Zookeeper Configuration ***********/ props.put(OffsetCommitRequiredAcksProp, offsetCommitRequiredAcks.toString) props.put(ProducerQuotaDefaultBytesPerSecondProp, producerQuotaDefaultBytesPerSecond.toString) props.put(ConsumerQuotaDefaultBytesPerSecondProp, consumerQuotaDefaultBytesPerSecond.toString) - props.put(ProducerQuotaOverridesProp, producerQuotaOverrides.toString) - props.put(ConsumerQuotaOverridesProp, consumerQuotaOverrides.toString) + props.put(ProducerQuotaBytesPerSecondOverridesProp, producerQuotaBytesPerSecondOverrides) + props.put(ConsumerQuotaBytesPerSecondOverridesProp, consumerQuotaBytesPerSecondOverrides) props.put(QuotaDelayFactorProp, quotaDelayFactor.toString) props.put(QuotaEnforcementBlackoutSecondsProp, quotaEnforcementBlackoutSeconds.toString) props.put(NumQuotaSamplesProp, numQuotaSamples.toString) props.put(QuotaWindowSizeSecondsProp, quotaWindowSizeSeconds.toString) + props.put(MetricsNumSamplesProp, metricsNumSamples.toString) + props.put(MetricsWindowSizeSecondsProp, metricsWindowSizeSeconds.toString) props.put(DeleteTopicEnableProp, deleteTopicEnable.toString) props.put(CompressionTypeProp, compressionType.toString) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 9f32b79..a4ffa58 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -26,6 +26,8 @@ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} import java.io.File +import org.apache.kafka.clients.producer.ProducerConfig + import collection.mutable import org.apache.kafka.common.metrics.{Metrics, JmxReporter, MetricsReporter, MetricConfig} @@ -189,12 +191,13 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg private def initMetrics() : Metrics = { val jmxPrefix: String = "kafka.server" - val reporters: Seq[MetricsReporter] = mutable.Seq(new JmxReporter(jmxPrefix)) - info("Initiated metrics") + info("Initiating KafkaMetrics") - new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), - JavaConversions.seqAsJavaList(reporters), - new org.apache.kafka.common.utils.SystemTime) + val metricConfig: MetricConfig = new MetricConfig().samples(config.metricsNumSamples) + .timeWindow(config.metricsWindowSizeSeconds, TimeUnit.SECONDS) + new Metrics(metricConfig, + JavaConversions.seqAsJavaList(mutable.Seq(new JmxReporter(jmxPrefix))), + new org.apache.kafka.common.utils.SystemTime) } private def initZk(): ZkClient = { diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index a604dd8..a09058e 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer._ import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.stats.Rate import org.apache.kafka.common.metrics.{Sensor, KafkaMetric, Metrics} import org.junit.Assert._ import org.junit.Test @@ -45,8 +46,7 @@ class QuotasTest extends KafkaServerTestHarness { overridingProps.put(KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp, "50000") overridingProps.put(KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp, "25000") // unthrottled - overridingProps.put(KafkaConfig.ProducerQuotaOverridesProp, producerId2 + "=" + Long.MaxValue) - overridingProps.put(KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp, "25000") + overridingProps.put(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp, producerId2 + "=" + Long.MaxValue) overridingProps.put(KafkaConfig.QuotaEnforcementBlackoutSecondsProp, "1") override def generateConfigs() = { diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala index 17b5973..f20f2a8 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala @@ -28,8 +28,8 @@ class ClientQuotaMetricsTest extends JUnit3Suite { private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), Collections.emptyList(), time) - private val config = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, - quotaOverrides = "p1=2000,p2=4000") + private val config = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000") var numCallbacks : Int = 0 def callback { @@ -88,23 +88,23 @@ class ClientQuotaMetricsTest extends JUnit3Suite { // Case 1 - Default config var testConfig = ClientQuotaMetricsConfig() var clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) - Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultQuotaPerSecond, true), + Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond, true), clientMetrics.quota("p1")) // Case 2 - Empty override - testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, - quotaOverrides = "p1=2000,p2=4000,,") + testConfig = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,,") clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) // Case 3 - NumberFormatException for override - testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, - quotaOverrides = "p1=2000,p2=4000,p3=p4") + testConfig = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,p3=p4") try { clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) - Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) + Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) } catch { // Swallow. @@ -112,11 +112,11 @@ class ClientQuotaMetricsTest extends JUnit3Suite { } // Case 4 - IllegalArgumentException for override - testConfig = ClientQuotaMetricsConfig(defaultQuotaPerSecond = 500, - quotaOverrides = "p1=2000=3000") + testConfig = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + quotaBytesPerSecondOverrides = "p1=2000=3000") try { clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "producer", time) - Assert.fail("Should fail to parse invalid config " + testConfig.quotaOverrides) + Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) } catch { // Swallow. diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala index 3a82844..b2345f0 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala @@ -220,12 +220,14 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.OffsetCommitTimeoutMsProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp => expected.setProperty(name, atLeastOneIntProp) - case KafkaConfig.ProducerQuotaOverridesProp => expected.setProperty(name, "P1=10,P2=20") - case KafkaConfig.ConsumerQuotaOverridesProp => expected.setProperty(name, "C1=10,C2=20") + case KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp => expected.setProperty(name, "P1=10,P2=20") + case KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp => expected.setProperty(name, "C1=10,C2=20") case KafkaConfig.QuotaDelayFactorProp => expected.setProperty(name, "%.1f".format(nextDouble + 1)) case KafkaConfig.QuotaEnforcementBlackoutSecondsProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.NumQuotaSamplesProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.QuotaWindowSizeSecondsProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.MetricsNumSamplesProp => expected.setProperty(name, atLeastOneIntProp) + case KafkaConfig.MetricsWindowSizeSecondsProp => expected.setProperty(name, atLeastOneIntProp) case KafkaConfig.DeleteTopicEnableProp => expected.setProperty(name, randFrom("true", "false")) // explicit, non trivial validations or with transient dependencies @@ -350,12 +352,14 @@ class KafkaConfigConfigDefTest extends JUnit3Suite { case KafkaConfig.OffsetCommitRequiredAcksProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-2") case KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") - case KafkaConfig.ProducerQuotaOverridesProp => // ignore string - case KafkaConfig.ConsumerQuotaOverridesProp => // ignore string + case KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp => // ignore string + case KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp => // ignore string case KafkaConfig.QuotaDelayFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.QuotaEnforcementBlackoutSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") case KafkaConfig.NumQuotaSamplesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.QuotaWindowSizeSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.MetricsNumSamplesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.MetricsWindowSizeSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0") case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") -- 1.7.12.4