From 87a65aaf8f844b5ee5cbd4ed384e0febbdf9ee4f Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 9 Apr 2015 17:18:37 -0700 Subject: [PATCH 01/21] 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. 6. This doesn't include a system test. There is a separate ticket for that 7. Fixed KAFKA-2191 - (Included fix from : https://reviews.apache.org/r/34418/ ) Signed-off-by: Aditya Auradkar --- .../org/apache/kafka/common/metrics/Quota.java | 24 +++ .../common/metrics/QuotaViolationException.java | 8 + .../org/apache/kafka/common/metrics/Sensor.java | 23 ++- .../apache/kafka/common/metrics/stats/Rate.java | 22 +- .../org/apache/kafka/common/utils/MockTime.java | 43 ++++ .../apache/kafka/common/metrics/MetricsTest.java | 28 ++- .../org/apache/kafka/common/utils/MockTime.java | 43 ---- .../scala/kafka/server/ClientQuotaMetrics.scala | 222 +++++++++++++++++++++ core/src/main/scala/kafka/server/KafkaApis.scala | 96 +++++++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 61 +++++- core/src/main/scala/kafka/server/KafkaServer.scala | 21 +- .../main/scala/kafka/server/ReplicaManager.scala | 4 +- .../main/scala/kafka/server/ThrottledRequest.scala | 46 +++++ .../scala/kafka/utils/ShutdownableThread.scala | 3 + .../scala/integration/kafka/api/QuotasTest.scala | 138 +++++++++++++ .../unit/kafka/server/ClientQuotaMetricsTest.scala | 150 ++++++++++++++ .../scala/unit/kafka/server/KafkaConfigTest.scala | 7 + .../server/ThrottledRequestExpirationTest.scala | 90 +++++++++ 18 files changed, 939 insertions(+), 90 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 create mode 100644 core/src/main/scala/kafka/server/ClientQuotaMetrics.scala create mode 100644 core/src/main/scala/kafka/server/ThrottledRequest.scala create mode 100644 core/src/test/scala/integration/kafka/api/QuotasTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.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/Quota.java b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java index d82bb0c..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,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..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 @@ -24,9 +24,17 @@ import org.apache.kafka.common.KafkaException; public class QuotaViolationException extends KafkaException { private static final long serialVersionUID = 1L; + 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 ca823fd..fb70391 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,30 @@ 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 (!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/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index 98429da..3ae9f7c 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -58,26 +58,28 @@ public class Rate implements MeasurableStat { @Override public double measure(MetricConfig config, long now) { double value = stat.measure(config, now); - double elapsed = convert(now - stat.oldest(now).lastWindowMs); - return value / elapsed; + // the elapsed time is always N-1 complete windows plus whatever fraction of the final window is complete + long elapsedCurrentWindowMs = now - stat.current(now).lastWindowMs; + long elapsedPriorWindowsMs = config.timeWindowMs() * (config.samples() - 1); + return value / convert(elapsedCurrentWindowMs + elapsedPriorWindowsMs); } - private double convert(long time) { + private double convert(long timeMs) { switch (unit) { case NANOSECONDS: - return time * 1000.0 * 1000.0; + return timeMs * 1000.0 * 1000.0; case MICROSECONDS: - return time * 1000.0; + return timeMs * 1000.0; case MILLISECONDS: - return time; + return timeMs; case SECONDS: - return time / 1000.0; + return timeMs / 1000.0; case MINUTES: - return time / (60.0 * 1000.0); + return timeMs / (60.0 * 1000.0); case HOURS: - return time / (60.0 * 60.0 * 1000.0); + return timeMs / (60.0 * 60.0 * 1000.0); case DAYS: - return time / (24.0 * 60.0 * 60.0 * 1000.0); + return timeMs / (24.0 * 60.0 * 60.0 * 1000.0); default: throw new IllegalStateException("Unknown unit: " + unit); } 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/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 544e120..8bddbb6 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -37,9 +37,9 @@ import org.junit.Test; public class MetricsTest { private static final double EPS = 0.000001; - - MockTime time = new MockTime(); - Metrics metrics = new Metrics(new MetricConfig(), Arrays.asList((MetricsReporter) new JmxReporter()), time); + private MockTime time = new MockTime(); + private MetricConfig config = new MetricConfig(); + private Metrics metrics = new Metrics(config, Arrays.asList((MetricsReporter) new JmxReporter()), time); @Test public void testMetricName() { @@ -77,19 +77,29 @@ public class MetricsTest { s2.add(new MetricName("s2.total", "grp1"), new Total()); s2.record(5.0); - for (int i = 0; i < 10; i++) + int sum = 0; + int count = 10; + for (int i = 0; i < count; i++) { s.record(i); + sum += i; + } + // prior to any time passing + double elapsedSecs = (config.timeWindowMs() * (config.samples() - 1)) / 1000.0; + assertEquals("Occurences(0...9) = 5", count / elapsedSecs, + metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); // pretend 2 seconds passed... - time.sleep(2000); + long sleepTime = 2; + time.sleep(sleepTime * 1000); + elapsedSecs += sleepTime; assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get(new MetricName("s2.total", "grp1")).value(), EPS); assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get(new MetricName("test.avg", "grp1")).value(), EPS); - assertEquals("Max(0...9) = 9", 9.0, metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS); + assertEquals("Max(0...9) = 9", count - 1, metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS); assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get(new MetricName("test.min", "grp1")).value(), EPS); - assertEquals("Rate(0...9) = 22.5", 22.5, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); - assertEquals("Occurences(0...9) = 5", 5.0, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); - assertEquals("Count(0...9) = 10", 10.0, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); + assertEquals("Rate(0...9) = 1.40625", sum / elapsedSecs, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); + assertEquals("Occurences(0...9) = 5", count / elapsedSecs, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); + assertEquals("Count(0...9) = 10", (double) count, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); } @Test 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 new file mode 100644 index 0000000..9067066 --- /dev/null +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -0,0 +1,222 @@ +/** + * 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.{DelayQueue, TimeUnit} + +import kafka.utils.{ShutdownableThread, Logging} +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics._ +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 + +/** + * Configuration settings for quota management + * @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 numQuotaSamples The number of sample to retain in memory + * @param quotaWindowSizeSeconds The time span of each sample + * + */ +case class ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond : Long = + ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond, + quotaBytesPerSecondOverrides : String = + ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides, + quotaDelayFactor: Double = + ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, + numQuotaSamples : Int = + ClientQuotaMetricsConfig.DefaultNumQuotaSamples, + quotaWindowSizeSeconds : Int = + ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) + +object ClientQuotaMetricsConfig { + val DefaultQuotaBytesPerSecond = Long.MaxValue + val QuotaBytesPerSecondOverrides = "" + val DefaultQuotaDelayFactor = 1 + // Always have 10 whole windows + 1 current window + val DefaultNumQuotaSamples = 11 + val DefaultQuotaWindowSizeSeconds = 1 + val MaxThrottleTimeSeconds = 30 +} + +/** + * 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, + private val apiKey : String, + private val time : Time) extends Logging { + 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) + 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, 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 + delayQueue.add(new ThrottledRequest(time, delayTime.toLong, callback)) + sensors._2.record(delayTime) + // If delayed, add the element to the delayQueue + logger.info("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), 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(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", + apiKey, + "Tracking throttle-time per client", + "client-id", + clientId), new Rate()) + 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) + .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 + } + + def shutdown() = { + throttledRequestReaper.shutdown() + } +} \ 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 7ea509c..349d8dc 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,6 +17,7 @@ package kafka.server +import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.TopicPartition import kafka.api._ @@ -42,9 +43,12 @@ 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) /** * Top-level method that handles all requests and multiplexes to the right api @@ -250,6 +254,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ 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]) { @@ -265,20 +270,32 @@ 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) + 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 { - requestChannel.noOperation(request.processor, request) + val response = ProducerResponse(produceRequest.correlationId, responseStatus) + requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, response))) } - } else { - val response = ProducerResponse(produceRequest.correlationId, responseStatus) - requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) + } + + 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)) } } @@ -316,14 +333,28 @@ 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(request.connectionId, response))) + def fetchResponseCallback { + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) + } + + // Do not throttle replication traffic + if (Request.isValidBrokerId(fetchRequest.replicaId)) { + fetchResponseCallback + } else { + val requestKey = RequestKeys.nameForKey(RequestKeys.FetchKey) + 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 @@ -604,9 +635,38 @@ 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( + defaultQuotaBytesPerSecond = cfg.producerQuotaDefaultBytesPerSecond, + quotaBytesPerSecondOverrides = cfg.producerQuotaBytesPerSecondOverrides, + quotaDelayFactor = cfg.quotaDelayFactor, + numQuotaSamples = cfg.numQuotaSamples, + quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds + ) + + val consumerQuotaManagerCfg = ClientQuotaMetricsConfig( + defaultQuotaBytesPerSecond = cfg.consumerQuotaDefaultBytesPerSecond, + quotaBytesPerSecondOverrides = cfg.consumerQuotaBytesPerSecondOverrides, + quotaDelayFactor = cfg.quotaDelayFactor, + numQuotaSamples = cfg.numQuotaSamples, + 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 org.apache.kafka.common.utils.SystemTime), + RequestKeys.nameForKey(RequestKeys.FetchKey) -> + new ClientQuotaMetrics(consumerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.FetchKey), new org.apache.kafka.common.utils.SystemTime) + ) + 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 - debug("Shut down complete.") + quotaManagers.foreach(entry => entry._2.shutdown()) + info("Shut down complete.") } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a06f0bd..ef5e67f 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -26,7 +26,10 @@ import kafka.consumer.ConsumerConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} +import org.apache.kafka.common.config.ConfigDef.Importance._ +import org.apache.kafka.common.config.ConfigDef.Range._ +import org.apache.kafka.common.config.ConfigDef.Type._ +import org.apache.kafka.common.config.{ConfigException, AbstractConfig, ConfigDef} import org.apache.kafka.common.metrics.MetricsReporter import org.apache.kafka.common.protocol.SecurityProtocol @@ -132,12 +135,22 @@ object Defaults { val OffsetCommitTimeoutMs = OffsetManagerConfig.DefaultOffsetCommitTimeoutMs val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks + /** ********* Quota Configuration ***********/ + val ProducerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond + val ConsumerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond + val ProducerQuotaBytesPerSecondOverrides = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides + val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides + val QuotaDelayFactor : Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor + val NumQuotaSamples : Int = ClientQuotaMetricsConfig.DefaultNumQuotaSamples + val QuotaWindowSizeSeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds + val DeleteTopicEnable = false val CompressionType = "producer" + /** ********* Kafka Metrics Configuration ***********/ val MetricNumSamples = 2 - val MetricSampleWindowMs = 1000 + val MetricSampleWindowMs = 30 val MetricReporterClasses = "" } @@ -250,15 +263,23 @@ 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 ProducerQuotaBytesPerSecondOverridesProp = "quota.producer.bytes.per.second.overrides" + val ConsumerQuotaBytesPerSecondOverridesProp = "quota.consumer.bytes.per.second.overrides" + val QuotaDelayFactorProp = "quota.delay.factor" + val NumQuotaSamplesProp = "quota.window.num" + val QuotaWindowSizeSecondsProp = "quota.window.size.seconds" val DeleteTopicEnableProp = "delete.topic.enable" val CompressionTypeProp = "compression.type" + /** ********* Kafka Metrics Configuration ***********/ val MetricSampleWindowMsProp = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG val MetricNumSamplesProp: String = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG val MetricReporterClassesProp: String = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG - /* Documentation */ /** ********* Zookeeper Configuration ***********/ val ZkConnectDoc = "Zookeeper host string" @@ -388,11 +409,23 @@ 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 ProducerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default producer quota. " + + "Example: clientIdX=10485760,clientIdY=10485760" + 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 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 " + "'producer' which means retain the original compression codec set by the producer." + /** ********* Kafka Metrics Configuration ***********/ val MetricSampleWindowMsDoc = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC val MetricNumSamplesDoc = CommonClientConfigs.METRICS_NUM_SAMPLES_DOC val MetricReporterClassesDoc = CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC @@ -518,9 +551,20 @@ object KafkaConfig { .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) .define(CompressionTypeProp, STRING, Defaults.CompressionType, HIGH, CompressionTypeDoc) + + /** ********* Kafka Metrics Configuration ***********/ .define(MetricNumSamplesProp, INT, Defaults.MetricNumSamples, atLeast(1), LOW, MetricNumSamplesDoc) .define(MetricSampleWindowMsProp, LONG, Defaults.MetricSampleWindowMs, atLeast(1), LOW, MetricSampleWindowMsDoc) .define(MetricReporterClassesProp, LIST, Defaults.MetricReporterClasses, LOW, MetricReporterClassesDoc) + + /** ********* Quota configuration ***********/ + .define(ProducerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ProducerQuotaDefaultBytesPerSecond, atLeast(1), HIGH, ProducerQuotaDefaultBytesPerSecondDoc) + .define(ConsumerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ConsumerQuotaDefaultBytesPerSecond, atLeast(1), HIGH, ConsumerQuotaDefaultBytesPerSecondDoc) + .define(ProducerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ProducerQuotaBytesPerSecondOverrides, HIGH, ProducerQuotaBytesPerSecondOverridesDoc) + .define(ConsumerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ConsumerQuotaBytesPerSecondOverrides, HIGH, ConsumerQuotaBytesPerSecondOverridesDoc) + .define(QuotaDelayFactorProp, DOUBLE, Defaults.QuotaDelayFactor, atLeast(1), LOW, QuotaDelayFactorDoc) + .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) + .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) } def configNames() = { @@ -548,7 +592,6 @@ object KafkaConfig { props.putAll(overrides) fromProps(props) } - } case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(KafkaConfig.configDef, props) { @@ -661,10 +704,18 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka val metricSampleWindowMs = getLong(KafkaConfig.MetricSampleWindowMsProp) val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) + /** ********* Quota Configuration **************/ + val producerQuotaDefaultBytesPerSecond = getLong(KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp) + val consumerQuotaDefaultBytesPerSecond = getLong(KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp) + val producerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp) + val consumerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp) + val quotaDelayFactor = getDouble(KafkaConfig.QuotaDelayFactorProp) + val numQuotaSamples = getInt(KafkaConfig.NumQuotaSamplesProp) + val quotaWindowSizeSeconds = getInt(KafkaConfig.QuotaWindowSizeSecondsProp) + val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) val compressionType = getString(KafkaConfig.CompressionTypeProp) - val listeners = getListeners val advertisedListeners = getAdvertisedListeners val logRetentionTimeMillis = getLogRetentionTimeMillis diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 84d4730..619eeff 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -32,6 +32,9 @@ import kafka.utils._ import org.apache.kafka.common.metrics._ import org.apache.kafka.common.network.NetworkReceive import org.apache.kafka.common.metrics.{JmxReporter, Metrics} +import org.apache.kafka.clients.producer.ProducerConfig + +import collection.mutable import scala.collection.mutable import org.I0Itec.zkclient.ZkClient @@ -92,7 +95,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" @@ -167,8 +170,13 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg consumerCoordinator.startup() /* start processing requests */ +<<<<<<< HEAD apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator, kafkaController, zkClient, config.brokerId, config, metadataCache) +======= + apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, + kafkaController, zkClient, config.brokerId, config, metadataCache, metrics) +>>>>>>> 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. 6. This doesn't include a system test. There is a separate ticket for that 7. Fixed KAFKA-2191 - (Included fix from : https://reviews.apache.org/r/34418/ ) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) @@ -208,6 +216,17 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg } } + private def initMetrics() : Metrics = { + val jmxPrefix: String = "kafka.server" + info("Initiating KafkaMetrics") + + 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 = { info("Connecting to zookeeper on " + config.zkConnect) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 2e0bbcd..d829e18 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -31,6 +31,7 @@ import kafka.metrics.KafkaMetricsGroup import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.metrics.Metrics import scala.collection._ @@ -98,7 +99,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 @@ -440,7 +441,6 @@ class ReplicaManager(val config: KafkaConfig, fetchMinBytes: Int, fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { - 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..dbbca36 --- /dev/null +++ b/core/src/main/scala/kafka/server/ThrottledRequest.scala @@ -0,0 +1,46 @@ +/** + * 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 org.apache.kafka.common.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 + + 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..a1524e5 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -0,0 +1,138 @@ +/** + * 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.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.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.KafkaMetric +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" + 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, "8000") + // un-throttled + overridingProps.put(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp, producerId2 + "=" + Long.MaxValue) + + 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 1 metric + Assert.assertEquals(1, 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 1 metric + Assert.assertEquals(1, numAsserts) + } + + def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): mutable.Map[MetricName, KafkaMetric] = { + 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 new file mode 100644 index 0000000..382b665 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala @@ -0,0 +1,150 @@ +/** + * 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.{Quota, MetricConfig} +import org.apache.kafka.common.utils.MockTime +import org.scalatest.junit.JUnit3Suite +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(), + time) + private val config = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + quotaBytesPerSecondOverrides = "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", time) + try { + 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")) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testQuotaViolation() { + val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer", time) + try { + /* 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, callback) + time.sleep(1000) + } + Assert.assertEquals(10, numCallbacks) + + // Create a spike. + // 400*10 + 2000 = 6000/10 = 600 bytes per second. + // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200 + val sleepTime = clientMetrics.record("unknown", 2000, callback) + Assert.assertEquals("Should be throttled", 2200, sleepTime) + // After a request is delayed, the callback cannot be triggered immediately + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(10, numCallbacks) + time.sleep(sleepTime) + + // Callback can only be triggered after the the delay time passes + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(11, numCallbacks) + + // Could continue to see delays until the bursty sample disappears + for(i <- 0 until 10) { + clientMetrics.record("unknown", 400, callback) + time.sleep(1000) + } + + Assert.assertEquals("Should be unthrottled since bursty sample has rolled over", + 0, clientMetrics.record("unknown", 0, callback)) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testOverrideParse() { + var testConfig = ClientQuotaMetricsConfig() + var clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) + + try { + // Case 1 - Default config + Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond, true), + clientMetrics.quota("p1")) + } finally { + clientMetrics.shutdown() + } + + + // Case 2 - Empty override + testConfig = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,,") + + clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) + try { + Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) + } finally { + clientMetrics.shutdown() + } + + // Case 3 - NumberFormatException for override + 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.quotaBytesPerSecondOverrides) + } + catch { + // Swallow. + case nfe : NumberFormatException => + } + + // Case 4 - IllegalArgumentException for override + 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.quotaBytesPerSecondOverrides) + } + catch { + // Swallow. + case nfe : IllegalArgumentException => + } + + } +} diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index e26a730..8ee7f9e 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -474,6 +474,13 @@ class KafkaConfigTest { 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.ProducerQuotaBytesPerSecondOverridesProp => // ignore string + case KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp => // ignore string + case KafkaConfig.QuotaDelayFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + 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") 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..1c88997 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala @@ -0,0 +1,90 @@ +/** + * 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 java.util.concurrent.{TimeUnit, DelayQueue} + +import org.apache.kafka.common.metrics.MetricConfig +import org.apache.kafka.common.utils.MockTime +import org.junit.{AfterClass, Before, Assert, Test} +import org.scalatest.junit.JUnit3Suite + +class ThrottledRequestExpirationTest extends JUnit3Suite { + 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) + + def callback { + numCallbacks += 1 + } + + @Before + def beforeMethod() { + numCallbacks = 0 + } + + @Test + def testExpire() { + val clientMetrics = new ClientQuotaMetrics(ClientQuotaMetricsConfig(), metrics, "producer", time) + + val delayQueue = new DelayQueue[ThrottledRequest]() + val reaper = new clientMetrics.ThrottledRequestReaper(delayQueue) + try { + // 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) + } finally { + clientMetrics.shutdown() + } + } + + @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 f545e6e61dde820ac35b8c3bae39372d17c38772 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 4 Jun 2015 16:31:03 -0700 Subject: [PATCH 02/21] Addressing Joel's comments --- .../scala/kafka/server/ClientQuotaMetrics.scala | 100 ++++++++++++--------- core/src/main/scala/kafka/server/KafkaApis.scala | 15 ++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 20 ++--- core/src/main/scala/kafka/server/KafkaServer.scala | 4 +- .../main/scala/kafka/server/ThrottledRequest.scala | 4 +- .../scala/kafka/utils/ShutdownableThread.scala | 2 +- .../scala/integration/kafka/api/QuotasTest.scala | 76 +++++++++++++--- .../unit/kafka/server/ClientQuotaMetricsTest.scala | 45 ++++++---- 8 files changed, 169 insertions(+), 97 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index 9067066..1f7b532 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -21,35 +21,40 @@ import java.util.concurrent.{DelayQueue, TimeUnit} import kafka.utils.{ShutdownableThread, Logging} import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ -import org.apache.kafka.common.metrics.stats.Rate +import org.apache.kafka.common.metrics.stats.{Total, Rate} import java.util.concurrent.locks.ReentrantReadWriteLock import org.apache.kafka.common.utils.Time -import scala.collection.mutable +/** + * Represents the sensors aggregated per client + * @param quotaSensor @Sensor that tracks the quota + * @param throttleTimeSensor @Sensor that tracks the throttle time + */ +private case class ClientSensors(quotaSensor: Sensor, throttleTimeSensor: Sensor) /** * Configuration settings for quota management - * @param defaultQuotaBytesPerSecond The default bytes per second quota allocated to any client + * @param quotaBytesPerSecondDefault 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 numQuotaSamples The number of sample to retain in memory + * @param numQuotaSamples The number of samples to retain in memory * @param quotaWindowSizeSeconds The time span of each sample * */ -case class ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond : Long = - ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond, - quotaBytesPerSecondOverrides : String = +case class ClientQuotaMetricsConfig(quotaBytesPerSecondDefault: Long = + ClientQuotaMetricsConfig.QuotaBytesPerSecondDefault, + quotaBytesPerSecondOverrides: String = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides, quotaDelayFactor: Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, - numQuotaSamples : Int = + numQuotaSamples: Int = ClientQuotaMetricsConfig.DefaultNumQuotaSamples, - quotaWindowSizeSeconds : Int = + quotaWindowSizeSeconds: Int = ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) object ClientQuotaMetricsConfig { - val DefaultQuotaBytesPerSecond = Long.MaxValue + val QuotaBytesPerSecondDefault = Long.MaxValue val QuotaBytesPerSecondOverrides = "" val DefaultQuotaDelayFactor = 1 // Always have 10 whole windows + 1 current window @@ -61,20 +66,27 @@ object ClientQuotaMetricsConfig { /** * 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 config @ClientQuotaMetricsConfig quota configs * @param metrics @Metrics Metrics instance + * @param apiKey API Key for the request + * @param time @Time object to use */ -class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, - private val metrics : Metrics, - private val apiKey : String, - private val time : Time) extends Logging { +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.quotaBytesPerSecondOverrides) - private val defaultQuota = Quota.lessThan(config.defaultQuotaBytesPerSecond) + private val defaultQuota = Quota.lessThan(config.quotaBytesPerSecondDefault) private val lock = new ReentrantReadWriteLock() private val delayQueue = new DelayQueue[ThrottledRequest]() val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) throttledRequestReaper.start() + private val delayQueueSensor = metrics.sensor(apiKey + "-delayQueue") + delayQueueSensor.add(new MetricName("queue-size", + apiKey, + "Tracks the size of the delay queue"), new Total()) + /** * Reaper thread that triggers callbacks on all throttled requests * @param delayQueue DelayQueue to dequeue from @@ -85,6 +97,8 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, override def doWork(): Unit = { val response : ThrottledRequest = delayQueue.poll(1, TimeUnit.SECONDS) if(response != null) { + // Decrement the size of the delay queue + delayQueueSensor.record(-1) trace("Response throttled for: " + response.delayTimeMs + " ms") response.execute() } @@ -92,7 +106,7 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, } /** - * Records that a produce request wrote some data + * Records that a clientId changed some metric being throttled (produced/consumed bytes, QPS etc.) * @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. @@ -100,20 +114,21 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, * @return Number of milliseconds to delay the response in case of Quota violation. * Zero otherwise */ - def record(clientId: String, value : Int, callback: => Unit) : Int = { - val sensors = getOrCreateQuotaSensors(clientId) + def recordAndMaybeThrottle(clientId: String, value: Int, callback: => Unit) : Int = { + val clientSensors = getOrCreateQuotaSensors(clientId) var delayTime = 0.0 try { - sensors._1.record(value) + clientSensors.quotaSensor.record(value) // trigger the callback immediately if quota is not violated callback } catch { - case qve : QuotaViolationException => - delayTime = qve.getDelayTimeMs*config.quotaDelayFactor + case qve: QuotaViolationException => + delayTime = qve.getDelayTimeMs * config.quotaDelayFactor delayQueue.add(new ThrottledRequest(time, delayTime.toLong, callback)) - sensors._2.record(delayTime) + delayQueueSensor.record() + clientSensors.throttleTimeSensor.record(delayTime) // If delayed, add the element to the delayQueue - logger.info("Quota violated for sensor (%s). Delay time: (%f)".format(sensors._1, delayTime), qve) + logger.debug("Quota violated for sensor (%s). Delay time: (%f)".format(clientSensors.quotaSensor.name(), delayTime), qve) } delayTime.toInt } @@ -122,18 +137,15 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, * Returns the consumer quota for the specified clientId * @return */ - private[server] def quota(clientId : String) : Quota = { - if(overriddenQuota.contains(clientId)) - overriddenQuota(clientId) - else - defaultQuota + private[server] def quota(clientId: String) : Quota = { + overriddenQuota.getOrElse(clientId, 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) = { + private def getOrCreateQuotaSensors(clientId: String) : ClientSensors = { // Names of the sensors to access val quotaSensorName = apiKey + "-" + clientId @@ -191,29 +203,31 @@ class ClientQuotaMetrics(private val config : ClientQuotaMetricsConfig, } } // return the read or created sensors - (quotaSensor, throttleTimeSensor) + ClientSensors(quotaSensor, throttleTimeSensor) } - private def getQuotaMetricConfig(quota : Quota) : MetricConfig = { + private def getQuotaMetricConfig(quota: Quota) : MetricConfig = { new MetricConfig() .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) .samples(config.numQuotaSamples) .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("")) { + /* Construct a Map of (clientId -> Quota) + * The input config is specified as a comma-separated K=V pairs + */ + private def initQuotaMap(input: String) : Map[String, Quota] = { + // If empty input, return an empty map + if (input.trim.length == 0) + Map[String, Quota]() + else + input.split(",").map(entry => { + val trimmedEntry = entry.trim 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 + throw new IllegalArgumentException("Incorrectly formatted override entry (%s). Format is k1=v1,k2=v2".format(entry)) + pair(0) -> new Quota(pair(1).toDouble, true) + }).toMap } def shutdown() = { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 349d8dc..70fe9db 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -293,9 +293,9 @@ class KafkaApis(val requestChannel: RequestChannel, val requestKey = RequestKeys.nameForKey(RequestKeys.ProduceKey) quotaManagers.get(requestKey) match { case Some(quotaManager) => - quotaManager.record(produceRequest.clientId, numBytesAppended, produceResponseCallback) + quotaManager.recordAndMaybeThrottle(produceRequest.clientId, numBytesAppended, produceResponseCallback) case None => - warn("Cannot throttle Api key %s".format(requestKey)) + warn("Cannot throttle producer request for clientId: %s".format(produceRequest.clientId)) } } @@ -344,13 +344,13 @@ class KafkaApis(val requestChannel: RequestChannel, } // Do not throttle replication traffic - if (Request.isValidBrokerId(fetchRequest.replicaId)) { + if (fetchRequest.isFromFollower) { fetchResponseCallback } else { val requestKey = RequestKeys.nameForKey(RequestKeys.FetchKey) quotaManagers.get(requestKey) match { case Some(quotaManager) => - quotaManager.record(fetchRequest.clientId, response.sizeInBytes, fetchResponseCallback) + quotaManager.recordAndMaybeThrottle(fetchRequest.clientId, response.sizeInBytes, fetchResponseCallback) case None => warn("Cannot throttle Api key %s".format(requestKey)) } @@ -638,9 +638,9 @@ class KafkaApis(val requestChannel: RequestChannel, /* * 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] = { + private def instantiateQuotaManagers(cfg: KafkaConfig): Map[String, ClientQuotaMetrics] = { val producerQuotaManagerCfg = ClientQuotaMetricsConfig( - defaultQuotaBytesPerSecond = cfg.producerQuotaDefaultBytesPerSecond, + quotaBytesPerSecondDefault = cfg.producerQuotaBytesPerSecondDefault, quotaBytesPerSecondOverrides = cfg.producerQuotaBytesPerSecondOverrides, quotaDelayFactor = cfg.quotaDelayFactor, numQuotaSamples = cfg.numQuotaSamples, @@ -648,14 +648,13 @@ class KafkaApis(val requestChannel: RequestChannel, ) val consumerQuotaManagerCfg = ClientQuotaMetricsConfig( - defaultQuotaBytesPerSecond = cfg.consumerQuotaDefaultBytesPerSecond, + quotaBytesPerSecondDefault = cfg.consumerQuotaBytesPerSecondDefault, quotaBytesPerSecondOverrides = cfg.consumerQuotaBytesPerSecondOverrides, quotaDelayFactor = cfg.quotaDelayFactor, numQuotaSamples = cfg.numQuotaSamples, 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 org.apache.kafka.common.utils.SystemTime), diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index ef5e67f..546cb31 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -136,8 +136,8 @@ object Defaults { val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks /** ********* Quota Configuration ***********/ - val ProducerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond - val ConsumerQuotaDefaultBytesPerSecond = ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond + val ProducerQuotaBytesPerSecondDefault = ClientQuotaMetricsConfig.QuotaBytesPerSecondDefault + val ConsumerQuotaBytesPerSecondDefault = ClientQuotaMetricsConfig.QuotaBytesPerSecondDefault val ProducerQuotaBytesPerSecondOverrides = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides val QuotaDelayFactor : Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor @@ -264,8 +264,8 @@ object KafkaConfig { 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 ProducerQuotaBytesPerSecondDefaultProp = "quota.producer.default" + val ConsumerQuotaBytesPerSecondDefaultProp = "quota.consumer.default" val ProducerQuotaBytesPerSecondOverridesProp = "quota.producer.bytes.per.second.overrides" val ConsumerQuotaBytesPerSecondOverridesProp = "quota.consumer.bytes.per.second.overrides" val QuotaDelayFactorProp = "quota.delay.factor" @@ -410,8 +410,8 @@ object KafkaConfig { "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 ProducerQuotaBytesPerSecondDefaultDoc = "Any producer distinguished by clientId will get throttled if it produces more bytes than this value per-second" + val ConsumerQuotaBytesPerSecondDefaultDoc = "Any consumer distinguished by clientId/consumer group will get throttled if it fetches more bytes than this value per-second" val ProducerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default producer quota. " + "Example: clientIdX=10485760,clientIdY=10485760" val ConsumerQuotaBytesPerSecondOverridesDoc = "Comma separated list of clientId:quotaBytesPerSecond to override the default consumer quota. " + @@ -558,8 +558,8 @@ object KafkaConfig { .define(MetricReporterClassesProp, LIST, Defaults.MetricReporterClasses, LOW, MetricReporterClassesDoc) /** ********* Quota configuration ***********/ - .define(ProducerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ProducerQuotaDefaultBytesPerSecond, atLeast(1), HIGH, ProducerQuotaDefaultBytesPerSecondDoc) - .define(ConsumerQuotaDefaultBytesPerSecondProp, LONG, Defaults.ConsumerQuotaDefaultBytesPerSecond, atLeast(1), HIGH, ConsumerQuotaDefaultBytesPerSecondDoc) + .define(ProducerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ProducerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ProducerQuotaBytesPerSecondDefaultDoc) + .define(ConsumerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ConsumerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ConsumerQuotaBytesPerSecondDefaultDoc) .define(ProducerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ProducerQuotaBytesPerSecondOverrides, HIGH, ProducerQuotaBytesPerSecondOverridesDoc) .define(ConsumerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ConsumerQuotaBytesPerSecondOverrides, HIGH, ConsumerQuotaBytesPerSecondOverridesDoc) .define(QuotaDelayFactorProp, DOUBLE, Defaults.QuotaDelayFactor, atLeast(1), LOW, QuotaDelayFactorDoc) @@ -705,8 +705,8 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) /** ********* Quota Configuration **************/ - val producerQuotaDefaultBytesPerSecond = getLong(KafkaConfig.ProducerQuotaDefaultBytesPerSecondProp) - val consumerQuotaDefaultBytesPerSecond = getLong(KafkaConfig.ConsumerQuotaDefaultBytesPerSecondProp) + val producerQuotaDefaultBytesPerSecond = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp) + val consumerQuotaDefaultBytesPerSecond = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp) val producerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp) val consumerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp) val quotaDelayFactor = getDouble(KafkaConfig.QuotaDelayFactorProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 619eeff..bb7b06d 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -217,10 +217,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg } private def initMetrics() : Metrics = { - val jmxPrefix: String = "kafka.server" + val jmxPrefix = "kafka.server" info("Initiating KafkaMetrics") - val metricConfig: MetricConfig = new MetricConfig().samples(config.metricsNumSamples) + val metricConfig = new MetricConfig().samples(config.metricsNumSamples) .timeWindow(config.metricsWindowSizeSeconds, TimeUnit.SECONDS) new Metrics(metricConfig, JavaConversions.seqAsJavaList(mutable.Seq(new JmxReporter(jmxPrefix))), diff --git a/core/src/main/scala/kafka/server/ThrottledRequest.scala b/core/src/main/scala/kafka/server/ThrottledRequest.scala index dbbca36..bf16bad 100644 --- a/core/src/main/scala/kafka/server/ThrottledRequest.scala +++ b/core/src/main/scala/kafka/server/ThrottledRequest.scala @@ -33,11 +33,11 @@ private[server] class ThrottledRequest(val time: Time, val delayTimeMs : Long, c def execute() = callback - def getDelay(unit: TimeUnit): Long = { + override def getDelay(unit: TimeUnit): Long = { unit.convert(endTime - time.milliseconds, TimeUnit.MILLISECONDS) } - def compareTo(d: Delayed): Int = { + override def compareTo(d: Delayed): Int = { val other = d.asInstanceOf[ThrottledRequest] if(this.endTime < other.endTime) -1 else if(this.endTime > other.endTime) 1 diff --git a/core/src/main/scala/kafka/utils/ShutdownableThread.scala b/core/src/main/scala/kafka/utils/ShutdownableThread.scala index db21aae..dc46797 100644 --- a/core/src/main/scala/kafka/utils/ShutdownableThread.scala +++ b/core/src/main/scala/kafka/utils/ShutdownableThread.scala @@ -52,7 +52,7 @@ abstract class ShutdownableThread(val name: String, val isInterruptible: Boolean } /** - * This method is continually in a loop called until the thread shuts down or this method throws an exception + * This method is repeatedly invoked until the thread shuts down or this method throws an exception */ def doWork(): Unit diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index a1524e5..fb05d9b 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -1,6 +1,4 @@ /** - * 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 @@ -22,13 +20,15 @@ import junit.framework.Assert import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.utils.TestUtils -import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.consumer.{ConsumerRecords, 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.KafkaMetric import org.junit.Assert._ import org.junit.Test +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable @@ -36,13 +36,19 @@ class QuotasTest extends KafkaServerTestHarness { private val producerBufferSize = 30000 private val producerId1 = "QuotasTestProducer-1" private val producerId2 = "QuotasTestProducer-2" + private val consumerId1 = "QuotasTestConsumer-1" + private val consumerId2 = "QuotasTestConsumer-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, "8000") + overridingProps.put(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp, "8000") + overridingProps.put(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp, "2500") + // un-throttled overridingProps.put(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp, producerId2 + "=" + Long.MaxValue) + overridingProps.put(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp, consumerId2 + "=" + Long.MaxValue) override def generateConfigs() = { FixedPortTestUtils.createBrokerConfigs(numServers, @@ -77,13 +83,21 @@ class QuotasTest extends KafkaServerTestHarness { val consumerProps = new Properties consumerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "QuotasTest") + consumerProps.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) 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) + consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range") + + //for(i <- 0 until 2) + consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId1) + consumers += new KafkaConsumer(consumerProps) + + consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId2) + consumers += new KafkaConsumer(consumerProps) val numPartitions = 1 val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) @@ -97,8 +111,11 @@ class QuotasTest extends KafkaServerTestHarness { } @Test - def testThrottledProducer() { - val allMetrics: mutable.Map[MetricName, KafkaMetric] = produce(producers(0), 1000) + def testThrottledProducerConsumer() { + val allMetrics: mutable.Map[MetricName, KafkaMetric] = servers.head.metrics.metrics().asScala + + val numRecords = 1000 + produce(producers(0), numRecords) var numAsserts = 0 for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { @@ -109,30 +126,63 @@ class QuotasTest extends KafkaServerTestHarness { } // Should have matched 1 metric Assert.assertEquals(1, numAsserts) + + // Consumer should read in a bursty manner and get throttled immediately + consume(consumers(0), numRecords) + + for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { + if (name.tags().containsValue(consumerId1) && name.name().startsWith("throttle-time")) { + Assert.assertTrue("Should have been throttled", metric.value() > 0) + numAsserts += 1 + } + } } @Test def testProducerOverrideUnthrottled() { - val allMetrics: mutable.Map[MetricName, KafkaMetric] = produce(producers(1), 1000) + val allMetrics: mutable.Map[MetricName, KafkaMetric] = servers.head.metrics.metrics().asScala + val numRecords = 1000 + + produce(producers(1), numRecords) 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) + Assert.assertEquals("Should not have been throttled", 0.0, metric.value()) numAsserts += 1 } } // Should have matched 1 metric Assert.assertEquals(1, numAsserts) + + consume(consumers(0), numRecords) + + for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { + if (name.tags().containsValue(consumerId2) && name.name().startsWith("throttle-time")) { + Assert.assertEquals("Should not have been throttled", 0.0, metric.value()) + numAsserts += 1 + } + } + } - def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): mutable.Map[MetricName, KafkaMetric] = { + def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int) { 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 } + + def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int) { + consumer.subscribe(topic1) + var numConsumed = 0 + while (numConsumed < numRecords) { + for (cr <- consumer.poll(100)) { + numConsumed += 1 + } + } + } + + } diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala index 382b665..d44f682 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala @@ -18,17 +18,17 @@ package kafka.server import java.util.Collections -import org.apache.kafka.common.metrics.{Quota, MetricConfig} +import kafka.api.RequestKeys +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.{Metrics, Quota, MetricConfig} import org.apache.kafka.common.utils.MockTime import org.scalatest.junit.JUnit3Suite 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(), - time) - private val config = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + + private val config = ClientQuotaMetricsConfig(quotaBytesPerSecondDefault = 500, quotaBytesPerSecondOverrides = "p1=2000,p2=4000") var numCallbacks : Int = 0 @@ -43,7 +43,7 @@ class ClientQuotaMetricsTest extends JUnit3Suite { @Test def testQuotaParsing() { - val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer", time) + val clientMetrics = new ClientQuotaMetrics(config, newMetrics, "producer", time) try { Assert.assertEquals("Default producer quota should be 500", new Quota(500, true), clientMetrics.quota("random-client-id")) @@ -58,22 +58,26 @@ class ClientQuotaMetricsTest extends JUnit3Suite { @Test def testQuotaViolation() { + val metrics = newMetrics val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer", time) + val queueSizeMetric = metrics.metrics().get(new MetricName("queue-size", "producer", "")) try { /* 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, callback) + clientMetrics.recordAndMaybeThrottle("unknown", 400, callback) time.sleep(1000) } Assert.assertEquals(10, numCallbacks) + Assert.assertEquals(0, queueSizeMetric.value().toInt) // Create a spike. // 400*10 + 2000 = 6000/10 = 600 bytes per second. // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200 - val sleepTime = clientMetrics.record("unknown", 2000, callback) + val sleepTime = clientMetrics.recordAndMaybeThrottle("unknown", 2000, callback) Assert.assertEquals("Should be throttled", 2200, sleepTime) + Assert.assertEquals(1, queueSizeMetric.value().toInt) // After a request is delayed, the callback cannot be triggered immediately clientMetrics.throttledRequestReaper.doWork() Assert.assertEquals(10, numCallbacks) @@ -81,16 +85,17 @@ class ClientQuotaMetricsTest extends JUnit3Suite { // Callback can only be triggered after the the delay time passes clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(0, queueSizeMetric.value().toInt) Assert.assertEquals(11, numCallbacks) // Could continue to see delays until the bursty sample disappears for(i <- 0 until 10) { - clientMetrics.record("unknown", 400, callback) + clientMetrics.recordAndMaybeThrottle("unknown", 400, callback) time.sleep(1000) } Assert.assertEquals("Should be unthrottled since bursty sample has rolled over", - 0, clientMetrics.record("unknown", 0, callback)) + 0, clientMetrics.recordAndMaybeThrottle("unknown", 0, callback)) } finally { clientMetrics.shutdown() } @@ -99,11 +104,11 @@ class ClientQuotaMetricsTest extends JUnit3Suite { @Test def testOverrideParse() { var testConfig = ClientQuotaMetricsConfig() - var clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) + var clientMetrics = new ClientQuotaMetrics(testConfig, newMetrics, "consumer", time) try { // Case 1 - Default config - Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.DefaultQuotaBytesPerSecond, true), + Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.QuotaBytesPerSecondDefault, true), clientMetrics.quota("p1")) } finally { clientMetrics.shutdown() @@ -111,10 +116,10 @@ class ClientQuotaMetricsTest extends JUnit3Suite { // Case 2 - Empty override - testConfig = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + testConfig = ClientQuotaMetricsConfig(quotaBytesPerSecondDefault = 500, quotaBytesPerSecondOverrides = "p1=2000,p2=4000,,") - clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) + clientMetrics = new ClientQuotaMetrics(testConfig, newMetrics, "consumer", time) try { Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) @@ -123,10 +128,10 @@ class ClientQuotaMetricsTest extends JUnit3Suite { } // Case 3 - NumberFormatException for override - testConfig = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + testConfig = ClientQuotaMetricsConfig(quotaBytesPerSecondDefault = 500, quotaBytesPerSecondOverrides = "p1=2000,p2=4000,p3=p4") try { - clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "consumer", time) + clientMetrics = new ClientQuotaMetrics(testConfig, newMetrics, "consumer", time) Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) } catch { @@ -135,10 +140,10 @@ class ClientQuotaMetricsTest extends JUnit3Suite { } // Case 4 - IllegalArgumentException for override - testConfig = ClientQuotaMetricsConfig(defaultQuotaBytesPerSecond = 500, + testConfig = ClientQuotaMetricsConfig(quotaBytesPerSecondDefault = 500, quotaBytesPerSecondOverrides = "p1=2000=3000") try { - clientMetrics = new ClientQuotaMetrics(testConfig, metrics, "producer", time) + clientMetrics = new ClientQuotaMetrics(testConfig, newMetrics, "producer", time) Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) } catch { @@ -147,4 +152,8 @@ class ClientQuotaMetricsTest extends JUnit3Suite { } } + + def newMetrics: Metrics = { + new Metrics(new MetricConfig(), Collections.emptyList(), time) + } } -- 1.7.12.4 From 5b891a3b1b129f878aa93be82b9a510e537f3333 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 8 Jun 2015 13:41:22 -0700 Subject: [PATCH 03/21] Minor imports changes --- core/src/test/scala/integration/kafka/api/QuotasTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index fb05d9b..9dc4690 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -20,7 +20,7 @@ import junit.framework.Assert import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.utils.TestUtils -import org.apache.kafka.clients.consumer.{ConsumerRecords, ConsumerConfig, KafkaConsumer} +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 -- 1.7.12.4 From a9621c13c17ea0644446acb59e521336e0e3e4ec Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 10 Jun 2015 10:48:12 -0700 Subject: [PATCH 04/21] Added testcase to verify that replication traffic is not throttled --- .../scala/integration/kafka/api/QuotasTest.scala | 23 +++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index 9dc4690..5d3c179 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -18,8 +18,8 @@ import java.util.Properties import junit.framework.Assert import kafka.integration.KafkaServerTestHarness -import kafka.server.KafkaConfig -import kafka.utils.TestUtils +import kafka.server.{KafkaServer, KafkaConfig} +import kafka.utils.{ZkUtils, TestUtils} import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer._ import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback @@ -39,7 +39,7 @@ class QuotasTest extends KafkaServerTestHarness { private val consumerId1 = "QuotasTestConsumer-1" private val consumerId2 = "QuotasTestConsumer-2" - val numServers = 1 + val numServers = 2 val overridingProps = new Properties() // Low enough quota that a producer sending a small payload in a tight loop should get throttled @@ -59,6 +59,7 @@ class QuotasTest extends KafkaServerTestHarness { var producers = mutable.Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() var consumers = mutable.Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + var leaderNode: KafkaServer = null private val topic1 = "topic-1" @@ -92,7 +93,6 @@ class QuotasTest extends KafkaServerTestHarness { classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range") - //for(i <- 0 until 2) consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId1) consumers += new KafkaConsumer(consumerProps) @@ -101,6 +101,7 @@ class QuotasTest extends KafkaServerTestHarness { val numPartitions = 1 val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) + leaderNode = if (leaders(0).get == servers(0).config.brokerId) servers(0) else servers(1) assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined)) } @@ -112,7 +113,7 @@ class QuotasTest extends KafkaServerTestHarness { @Test def testThrottledProducerConsumer() { - val allMetrics: mutable.Map[MetricName, KafkaMetric] = servers.head.metrics.metrics().asScala + val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala val numRecords = 1000 produce(producers(0), numRecords) @@ -136,11 +137,14 @@ class QuotasTest extends KafkaServerTestHarness { numAsserts += 1 } } + + // This tests that replica traffic does not get throttled + assertNoUnderReplicatedPartition } @Test def testProducerOverrideUnthrottled() { - val allMetrics: mutable.Map[MetricName, KafkaMetric] = servers.head.metrics.metrics().asScala + val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala val numRecords = 1000 produce(producers(1), numRecords) @@ -164,6 +168,8 @@ class QuotasTest extends KafkaServerTestHarness { } } + // This tests that replica traffic does not get throttled + assertNoUnderReplicatedPartition } def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int) { @@ -184,5 +190,8 @@ class QuotasTest extends KafkaServerTestHarness { } } - + def assertNoUnderReplicatedPartition() = { + val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic1, 0) + Assert.assertEquals("Should have 2 replicas in ISR", numServers, inSyncReplicas.size) + } } -- 1.7.12.4 From 8c7fe27cb73fe04c028ed18bb9525df5f4299da6 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 11 Jun 2015 17:32:14 -0700 Subject: [PATCH 05/21] Tmp commit --- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/main/scala/kafka/server/KafkaServer.scala | 39 +++++----------------- .../integration/kafka/api/ConsumerBounceTest.scala | 2 +- .../scala/integration/kafka/api/QuotasTest.scala | 14 ++++---- 4 files changed, 17 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 546cb31..95d3df2 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -150,7 +150,7 @@ object Defaults { /** ********* Kafka Metrics Configuration ***********/ val MetricNumSamples = 2 - val MetricSampleWindowMs = 30 + val MetricSampleWindowMs = 30000 val MetricReporterClasses = "" } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index bb7b06d..6fee2ff 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -18,7 +18,6 @@ package kafka.server import java.util -import java.util.Properties import kafka.admin._ import kafka.log.LogConfig @@ -32,9 +31,6 @@ import kafka.utils._ import org.apache.kafka.common.metrics._ import org.apache.kafka.common.network.NetworkReceive import org.apache.kafka.common.metrics.{JmxReporter, Metrics} -import org.apache.kafka.clients.producer.ProducerConfig - -import collection.mutable import scala.collection.mutable import org.I0Itec.zkclient.ZkClient @@ -65,11 +61,11 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg private val reporters: java.util.List[MetricsReporter] = config.metricReporterClasses reporters.add(new JmxReporter(jmxPrefix)) - - - // This exists so SocketServer (which uses Client libraries) can use the client Time objects without having to convert all of Kafka to use them - // Once we get rid of kafka.utils.time, we can get rid of this too - private val socketServerTime: org.apache.kafka.common.utils.Time = new org.apache.kafka.common.utils.SystemTime() + // This exists because the Metrics package from clients has its own Time implementation. + // SocketServer/Quotas (which uses client libraries) have to use the client Time objects without having to convert all of Kafka to use them + // Eventually, we want to merge the Time objects in core and clients + private val kafkaMetricsTime: org.apache.kafka.common.utils.Time = new org.apache.kafka.common.utils.SystemTime() + val metrics = new Metrics(metricConfig, reporters, kafkaMetricsTime) val brokerState: BrokerState = new BrokerState @@ -83,7 +79,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var dynamicConfigHandlers: Map[String, ConfigHandler] = null var dynamicConfigManager: DynamicConfigManager = null - val metrics: Metrics = new Metrics() var consumerCoordinator: ConsumerCoordinator = null @@ -95,7 +90,6 @@ 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" @@ -140,9 +134,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.brokerId = getBrokerId this.logIdent = "[Kafka Server " + config.brokerId + "], " - val metrics = new Metrics(metricConfig, reporters, socketServerTime) - - socketServer = new SocketServer(config.brokerId, config.listeners, config.numNetworkThreads, @@ -153,7 +144,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.maxConnectionsPerIp, config.connectionsMaxIdleMs, config.maxConnectionsPerIpOverrides, - socketServerTime, + kafkaMetricsTime, metrics) socketServer.startup() @@ -170,13 +161,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg consumerCoordinator.startup() /* start processing requests */ -<<<<<<< HEAD apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator, - kafkaController, zkClient, config.brokerId, config, metadataCache) -======= - apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, kafkaController, zkClient, config.brokerId, config, metadataCache, metrics) ->>>>>>> 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. 6. This doesn't include a system test. There is a separate ticket for that 7. Fixed KAFKA-2191 - (Included fix from : https://reviews.apache.org/r/34418/ ) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) @@ -216,17 +202,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg } } - private def initMetrics() : Metrics = { - val jmxPrefix = "kafka.server" - info("Initiating KafkaMetrics") - - val 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 = { info("Connecting to zookeeper on " + config.zkConnect) @@ -381,6 +356,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg CoreUtils.swallow(kafkaController.shutdown()) if(zkClient != null) CoreUtils.swallow(zkClient.close()) + if(metrics != null) + CoreUtils.swallow(metrics.close()) brokerState.newState(NotRunning) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index cf65f12..6de61f3 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -96,7 +96,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { scheduler.shutdown() } - def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) + //def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) def seekAndCommitWithBrokerFailures(numIters: Int) { val numRecords = 1000 diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index 5d3c179..d570c3a 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -33,7 +33,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable class QuotasTest extends KafkaServerTestHarness { - private val producerBufferSize = 30000 + private val producerBufferSize = 300000 private val producerId1 = "QuotasTestProducer-1" private val producerId2 = "QuotasTestProducer-2" private val consumerId1 = "QuotasTestConsumer-1" @@ -101,7 +101,7 @@ class QuotasTest extends KafkaServerTestHarness { val numPartitions = 1 val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) - leaderNode = if (leaders(0).get == servers(0).config.brokerId) servers(0) else servers(1) + leaderNode = if (leaders(0).get == servers.head.config.brokerId) servers.head else servers(1) assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined)) } @@ -116,7 +116,7 @@ class QuotasTest extends KafkaServerTestHarness { val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala val numRecords = 1000 - produce(producers(0), numRecords) + produce(producers.head, numRecords) var numAsserts = 0 for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { @@ -129,7 +129,7 @@ class QuotasTest extends KafkaServerTestHarness { Assert.assertEquals(1, numAsserts) // Consumer should read in a bursty manner and get throttled immediately - consume(consumers(0), numRecords) + consume(consumers.head, numRecords) for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { if (name.tags().containsValue(consumerId1) && name.name().startsWith("throttle-time")) { @@ -159,7 +159,7 @@ class QuotasTest extends KafkaServerTestHarness { // Should have matched 1 metric Assert.assertEquals(1, numAsserts) - consume(consumers(0), numRecords) + consume(consumers.head, numRecords) for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { if (name.tags().containsValue(consumerId2) && name.name().startsWith("throttle-time")) { @@ -169,7 +169,7 @@ class QuotasTest extends KafkaServerTestHarness { } // This tests that replica traffic does not get throttled - assertNoUnderReplicatedPartition + assertNoUnderReplicatedPartition() } def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int) { @@ -190,7 +190,7 @@ class QuotasTest extends KafkaServerTestHarness { } } - def assertNoUnderReplicatedPartition() = { + def assertNoUnderReplicatedPartition() { val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic1, 0) Assert.assertEquals("Should have 2 replicas in ISR", numServers, inSyncReplicas.size) } -- 1.7.12.4 From 3fc66cc3939758058ee24ed3ed37e1370bb967ae Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 11 Jun 2015 18:03:43 -0700 Subject: [PATCH 06/21] Fixing test failure --- core/src/main/scala/kafka/server/KafkaServer.scala | 4 +++- core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 6fee2ff..c92369d 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -65,7 +65,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg // SocketServer/Quotas (which uses client libraries) have to use the client Time objects without having to convert all of Kafka to use them // Eventually, we want to merge the Time objects in core and clients private val kafkaMetricsTime: org.apache.kafka.common.utils.Time = new org.apache.kafka.common.utils.SystemTime() - val metrics = new Metrics(metricConfig, reporters, kafkaMetricsTime) + var metrics: Metrics = null val brokerState: BrokerState = new BrokerState @@ -118,6 +118,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg val canStartup = isStartingUp.compareAndSet(false, true) if (canStartup) { + metrics = new Metrics(metricConfig, reporters, kafkaMetricsTime) + brokerState.newState(Starting) /* start scheduler */ diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 6de61f3..cf65f12 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -96,7 +96,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { scheduler.shutdown() } - //def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) + def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(5) def seekAndCommitWithBrokerFailures(numIters: Int) { val numRecords = 1000 -- 1.7.12.4 From 2a7f64bdbad3fb9d56e79eefe0b65e3138dc957f Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Fri, 12 Jun 2015 10:37:46 -0700 Subject: [PATCH 07/21] Minor --- core/src/test/scala/integration/kafka/api/QuotasTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index d570c3a..245852b 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -139,7 +139,7 @@ class QuotasTest extends KafkaServerTestHarness { } // This tests that replica traffic does not get throttled - assertNoUnderReplicatedPartition + assertNoUnderReplicatedPartition() } @Test -- 1.7.12.4 From d81d1275aba6e081fd59687f34895fe117fed774 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 29 Jun 2015 17:53:05 -0700 Subject: [PATCH 08/21] Addressing Joel's comments --- .../common/metrics/QuotaViolationException.java | 5 -- .../org/apache/kafka/common/metrics/Sensor.java | 2 +- .../apache/kafka/common/metrics/MetricsTest.java | 18 +++++--- .../scala/kafka/server/ClientQuotaMetrics.scala | 4 +- .../main/scala/kafka/server/ThrottledRequest.scala | 4 +- .../scala/integration/kafka/api/QuotasTest.scala | 53 ++++++++++++++-------- 6 files changed, 49 insertions(+), 37 deletions(-) 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 dcc415c..08454e8 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 @@ -26,15 +26,10 @@ public class QuotaViolationException extends KafkaException { private static final long serialVersionUID = 1L; 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 fb70391..b20417d 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 @@ -117,7 +117,7 @@ public final class Sensor { double value = metric.value(timeMs); if (!quota.acceptable(value)) { throw new QuotaViolationException(String.format( - "(%s) is violating its quota of (%f) with value (%f)", + "(%s) violated quota. Actual: (%f), Threshold: (%f)", metric.metricName(), quota.bound(), value), diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 8bddbb6..d5d0281 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -85,21 +85,25 @@ public class MetricsTest { } // prior to any time passing double elapsedSecs = (config.timeWindowMs() * (config.samples() - 1)) / 1000.0; - assertEquals("Occurences(0...9) = 5", count / elapsedSecs, + assertEquals(String.format("Occurrences(0...%d) = %d", count, count/elapsedSecs), count / elapsedSecs, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); // pretend 2 seconds passed... - long sleepTime = 2; - time.sleep(sleepTime * 1000); - elapsedSecs += sleepTime; + long sleepTimeMs = 2; + time.sleep(sleepTimeMs * 1000); + elapsedSecs += sleepTimeMs; assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get(new MetricName("s2.total", "grp1")).value(), EPS); assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get(new MetricName("test.avg", "grp1")).value(), EPS); assertEquals("Max(0...9) = 9", count - 1, metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS); assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get(new MetricName("test.min", "grp1")).value(), EPS); - assertEquals("Rate(0...9) = 1.40625", sum / elapsedSecs, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); - assertEquals("Occurences(0...9) = 5", count / elapsedSecs, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); - assertEquals("Count(0...9) = 10", (double) count, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); + assertEquals("Rate(0...9) = 1.40625", + sum / elapsedSecs, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); + assertEquals(String.format("Occurrences(0...%d) = %d", count, count/elapsedSecs), + count / elapsedSecs, + metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); + assertEquals("Count(0...9) = 10", + (double) count, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); } @Test diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala index 1f7b532..9bb8fd1 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala @@ -137,9 +137,7 @@ class ClientQuotaMetrics(private val config: ClientQuotaMetricsConfig, * Returns the consumer quota for the specified clientId * @return */ - private[server] def quota(clientId: String) : Quota = { - overriddenQuota.getOrElse(clientId, defaultQuota) - } + private[server] def quota(clientId: String) : Quota = overriddenQuota.getOrElse(clientId, defaultQuota) /* * This function either returns the sensors for a given client id or creates them if they don't exist diff --git a/core/src/main/scala/kafka/server/ThrottledRequest.scala b/core/src/main/scala/kafka/server/ThrottledRequest.scala index bf16bad..103a951 100644 --- a/core/src/main/scala/kafka/server/ThrottledRequest.scala +++ b/core/src/main/scala/kafka/server/ThrottledRequest.scala @@ -39,8 +39,8 @@ private[server] class ThrottledRequest(val time: Time, val delayTimeMs : Long, c override def compareTo(d: Delayed): Int = { val other = d.asInstanceOf[ThrottledRequest] - if(this.endTime < other.endTime) -1 - else if(this.endTime > other.endTime) 1 + if (this.endTime < other.endTime) -1 + else if (this.endTime > other.endTime) 1 else 0 } } diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index 245852b..44daa9a 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -17,6 +17,7 @@ package kafka.api import java.util.Properties import junit.framework.Assert +import kafka.consumer.SimpleConsumer import kafka.integration.KafkaServerTestHarness import kafka.server.{KafkaServer, KafkaConfig} import kafka.utils.{ZkUtils, TestUtils} @@ -59,8 +60,10 @@ class QuotasTest extends KafkaServerTestHarness { var producers = mutable.Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() var consumers = mutable.Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() - var leaderNode: KafkaServer = null + var replicaConsumers = mutable.Buffer[SimpleConsumer]() + var leaderNode: KafkaServer = null + var followerNode: KafkaServer = null private val topic1 = "topic-1" override def setUp() { @@ -80,6 +83,12 @@ class QuotasTest extends KafkaServerTestHarness { producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, producerId2) producers += new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + val numPartitions = 1 + val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) + leaderNode = if (leaders(0).get == servers.head.config.brokerId) servers.head else servers(1) + followerNode = if (leaders(0).get != servers.head.config.brokerId) servers.head else servers(1) + assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined)) + // Create consumers val consumerProps = new Properties consumerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) @@ -95,19 +104,19 @@ class QuotasTest extends KafkaServerTestHarness { consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId1) consumers += new KafkaConsumer(consumerProps) + // Create replica consumers with the same clientId as the high level consumer. These requests should never be throttled + replicaConsumers += new SimpleConsumer("localhost", leaderNode.boundPort(), 1000000, 64*1024, consumerId1) consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId2) consumers += new KafkaConsumer(consumerProps) + replicaConsumers += new SimpleConsumer("localhost", leaderNode.boundPort(), 1000000, 64*1024, consumerId2) - val numPartitions = 1 - val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers) - leaderNode = if (leaders(0).get == servers.head.config.brokerId) servers.head else servers(1) - 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 ) + replicaConsumers.foreach( _.close ) super.tearDown() } @@ -116,7 +125,9 @@ class QuotasTest extends KafkaServerTestHarness { val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala val numRecords = 1000 - produce(producers.head, numRecords) + val startTime = System.currentTimeMillis() + val bytesProduced = produce(producers.head, numRecords) + val elapsedTime = System.currentTimeMillis() - startTime var numAsserts = 0 for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { @@ -130,6 +141,11 @@ class QuotasTest extends KafkaServerTestHarness { // Consumer should read in a bursty manner and get throttled immediately consume(consumers.head, numRecords) + // The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately + val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build() + val fetchResponse: FetchResponse = replicaConsumers.head.fetch(request) + // fetchResponse.throttleTime should be 0 + // Assert.assertEquals("Throttle Time should be 0", 0, fetchResponse.throttleTime) for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { if (name.tags().containsValue(consumerId1) && name.name().startsWith("throttle-time")) { @@ -137,9 +153,6 @@ class QuotasTest extends KafkaServerTestHarness { numAsserts += 1 } } - - // This tests that replica traffic does not get throttled - assertNoUnderReplicatedPartition() } @Test @@ -159,7 +172,13 @@ class QuotasTest extends KafkaServerTestHarness { // Should have matched 1 metric Assert.assertEquals(1, numAsserts) + // The "client" consumer does not get throttled. consume(consumers.head, numRecords) + // The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately + val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build() + val fetchResponse: FetchResponse = replicaConsumers(1).fetch(request) + // fetchResponse.throttleTime should be 0 + // Assert.assertEquals("Throttle Time should be 0", 0, fetchResponse.throttleTime) for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { if (name.tags().containsValue(consumerId2) && name.name().startsWith("throttle-time")) { @@ -167,17 +186,18 @@ class QuotasTest extends KafkaServerTestHarness { numAsserts += 1 } } - - // This tests that replica traffic does not get throttled - assertNoUnderReplicatedPartition() } - def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int) { + def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): Int = { + var numBytesProduced = 0 for (i <- 0 to count) { - p.send(new ProducerRecord[Array[Byte], Array[Byte]](topic1, null, null, i.toString.getBytes), + val payload = i.toString.getBytes + numBytesProduced += payload.length + p.send(new ProducerRecord[Array[Byte], Array[Byte]](topic1, null, null, payload), new ErrorLoggingCallback(topic1, null, null, true)).get() Thread.sleep(1) } + numBytesProduced } def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int) { @@ -189,9 +209,4 @@ class QuotasTest extends KafkaServerTestHarness { } } } - - def assertNoUnderReplicatedPartition() { - val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic1, 0) - Assert.assertEquals("Should have 2 replicas in ISR", numServers, inSyncReplicas.size) - } } -- 1.7.12.4 From cc8625bf5d082e5442c08798402d101648a0093a Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 4 Aug 2015 18:23:50 -0700 Subject: [PATCH 09/21] Addressing comments --- .../common/metrics/QuotaViolationException.java | 2 +- .../org/apache/kafka/common/metrics/Sensor.java | 6 +- .../scala/kafka/server/ClientQuotaManager.scala | 234 +++++++++++++++++++++ .../scala/kafka/server/ClientQuotaMetrics.scala | 234 --------------------- core/src/main/scala/kafka/server/KafkaApis.scala | 30 +-- core/src/main/scala/kafka/server/KafkaConfig.scala | 14 +- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- .../main/scala/kafka/server/ThrottledRequest.scala | 46 ---- .../scala/kafka/server/ThrottledResponse.scala | 46 ++++ .../scala/integration/kafka/api/QuotasTest.scala | 74 +++---- .../unit/kafka/server/ClientQuotaManagerTest.scala | 159 ++++++++++++++ .../unit/kafka/server/ClientQuotaMetricsTest.scala | 159 -------------- .../server/ThrottledRequestExpirationTest.scala | 90 -------- .../server/ThrottledResponseExpirationTest.scala | 90 ++++++++ 14 files changed, 583 insertions(+), 603 deletions(-) create mode 100644 core/src/main/scala/kafka/server/ClientQuotaManager.scala delete mode 100644 core/src/main/scala/kafka/server/ClientQuotaMetrics.scala delete mode 100644 core/src/main/scala/kafka/server/ThrottledRequest.scala create mode 100644 core/src/main/scala/kafka/server/ThrottledResponse.scala create mode 100644 core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala delete mode 100644 core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala delete mode 100644 core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala 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 08454e8..742fc14 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,7 +24,7 @@ import org.apache.kafka.common.KafkaException; public class QuotaViolationException extends KafkaException { private static final long serialVersionUID = 1L; - private int delayTimeMs; + private final int delayTimeMs; public QuotaViolationException(String m, int delayTimeMs) { super(m); 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 b20417d..901741f 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 @@ -121,7 +121,7 @@ public final class Sensor { metric.metricName(), quota.bound(), value), - delayTime(metric, timeMs, quota, value, config)); + delayTime(quota, value, config)); } } } @@ -132,10 +132,10 @@ public final class Sensor { * 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) { + private int delayTime(Quota quota, double metricValue, MetricConfig config) { double difference = metricValue - quota.bound(); double time = difference / quota.bound() * config.samples() * config.timeWindowMs(); - return (int) time; + return (int) Math.round(time); } /** diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala new file mode 100644 index 0000000..66a66f5 --- /dev/null +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -0,0 +1,234 @@ +/** + * 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.{DelayQueue, TimeUnit} + +import kafka.utils.{ShutdownableThread, Logging} +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics._ +import org.apache.kafka.common.metrics.stats.{Total, Rate} +import java.util.concurrent.locks.ReentrantReadWriteLock + +import org.apache.kafka.common.utils.Time + +/** + * Represents the sensors aggregated per client + * @param quotaSensor @Sensor that tracks the quota + * @param throttleTimeSensor @Sensor that tracks the throttle time + */ +private case class ClientSensors(quotaSensor: Sensor, throttleTimeSensor: Sensor) + +/** + * Configuration settings for quota management + * @param quotaBytesPerSecondDefault 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 numQuotaSamples The number of samples to retain in memory + * @param quotaWindowSizeSeconds The time span of each sample + * + */ +case class ClientQuotaManagerConfig(quotaBytesPerSecondDefault: Long = + ClientQuotaManagerConfig.QuotaBytesPerSecondDefault, + quotaBytesPerSecondOverrides: String = + ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides, + quotaDelayFactor: Double = + ClientQuotaManagerConfig.DefaultQuotaDelayFactor, + numQuotaSamples: Int = + ClientQuotaManagerConfig.DefaultNumQuotaSamples, + quotaWindowSizeSeconds: Int = + ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds) + +object ClientQuotaManagerConfig { + val QuotaBytesPerSecondDefault = Long.MaxValue + val QuotaBytesPerSecondOverrides = "" + val DefaultQuotaDelayFactor = 1 + // Always have 10 whole windows + 1 current window + val DefaultNumQuotaSamples = 11 + val DefaultQuotaWindowSizeSeconds = 1 + val MaxThrottleTimeSeconds = 30 +} + +/** + * Helper class that records per-client metrics. It is also responsible for maintaining Quota usage statistics + * for all clients. + * @param config @ClientQuotaMetricsConfig quota configs + * @param metrics @Metrics Metrics instance + * @param apiKey API Key for the request + * @param time @Time object to use + */ +class ClientQuotaManager(private val config: ClientQuotaManagerConfig, + private val metrics: Metrics, + private val apiKey: String, + private val time: Time) extends Logging { + private val overriddenQuota = initQuotaMap(config.quotaBytesPerSecondOverrides) + private val defaultQuota = Quota.lessThan(config.quotaBytesPerSecondDefault) + private val lock = new ReentrantReadWriteLock() + private val delayQueue = new DelayQueue[ThrottledResponse]() + val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) + throttledRequestReaper.start() + + private val delayQueueSensor = metrics.sensor(apiKey + "-delayQueue") + delayQueueSensor.add(new MetricName("queue-size", + apiKey, + "Tracks the size of the delay queue"), new Total()) + + /** + * Reaper thread that triggers callbacks on all throttled requests + * @param delayQueue DelayQueue to dequeue from + */ + class ThrottledRequestReaper(delayQueue: DelayQueue[ThrottledResponse]) extends ShutdownableThread( + "ThrottledRequestReaper-%s".format(apiKey), false) { + + override def doWork(): Unit = { + val response: ThrottledResponse = delayQueue.poll(1, TimeUnit.SECONDS) + if (response != null) { + // Decrement the size of the delay queue + delayQueueSensor.record(-1) + trace("Response throttled for: " + response.delayTimeMs + " ms") + response.execute() + } + } + } + + /** + * Records that a clientId changed some metric being throttled (produced/consumed bytes, QPS etc.) + * @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 recordAndMaybeThrottle(clientId: String, value: Int, callback: => Unit): Int = { + val clientSensors = getOrCreateQuotaSensors(clientId) + var delayTime = 0.0 + try { + clientSensors.quotaSensor.record(value) + // trigger the callback immediately if quota is not violated + callback + } catch { + case qve: QuotaViolationException => + delayTime = qve.getDelayTimeMs * config.quotaDelayFactor + delayQueue.add(new ThrottledResponse(time, delayTime.toLong, callback)) + delayQueueSensor.record() + clientSensors.throttleTimeSensor.record(delayTime) + // If delayed, add the element to the delayQueue + logger.debug("Quota violated for sensor (%s). Delay time: (%f)".format(clientSensors.quotaSensor.name(), delayTime), qve) + } + delayTime.toInt + } + + /** + * Returns the consumer quota for the specified clientId + * @return + */ + private[server] def quota(clientId: String): Quota = overriddenQuota.getOrElse(clientId, 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): ClientSensors = { + + // 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", + apiKey, + "Tracking throttle-time per client", + "client-id", + clientId), new Rate()) + 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 + ClientSensors(quotaSensor, throttleTimeSensor) + } + + private def getQuotaMetricConfig(quota: Quota): MetricConfig = { + new MetricConfig() + .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) + .samples(config.numQuotaSamples) + .quota(quota) + } + + /* Construct a Map of (clientId -> Quota) + * The input config is specified as a comma-separated K=V pairs + */ + private def initQuotaMap(input: String): Map[String, Quota] = { + // If empty input, return an empty map + if (input.trim.length == 0) + Map[String, Quota]() + else + input.split(",").map(entry => { + val trimmedEntry = entry.trim + 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)) + pair(0) -> new Quota(pair(1).toDouble, true) + }).toMap + } + + def shutdown() = { + throttledRequestReaper.shutdown() + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala b/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala deleted file mode 100644 index 9bb8fd1..0000000 --- a/core/src/main/scala/kafka/server/ClientQuotaMetrics.scala +++ /dev/null @@ -1,234 +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.{DelayQueue, TimeUnit} - -import kafka.utils.{ShutdownableThread, Logging} -import org.apache.kafka.common.MetricName -import org.apache.kafka.common.metrics._ -import org.apache.kafka.common.metrics.stats.{Total, Rate} -import java.util.concurrent.locks.ReentrantReadWriteLock - -import org.apache.kafka.common.utils.Time - -/** - * Represents the sensors aggregated per client - * @param quotaSensor @Sensor that tracks the quota - * @param throttleTimeSensor @Sensor that tracks the throttle time - */ -private case class ClientSensors(quotaSensor: Sensor, throttleTimeSensor: Sensor) - -/** - * Configuration settings for quota management - * @param quotaBytesPerSecondDefault 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 numQuotaSamples The number of samples to retain in memory - * @param quotaWindowSizeSeconds The time span of each sample - * - */ -case class ClientQuotaMetricsConfig(quotaBytesPerSecondDefault: Long = - ClientQuotaMetricsConfig.QuotaBytesPerSecondDefault, - quotaBytesPerSecondOverrides: String = - ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides, - quotaDelayFactor: Double = - ClientQuotaMetricsConfig.DefaultQuotaDelayFactor, - numQuotaSamples: Int = - ClientQuotaMetricsConfig.DefaultNumQuotaSamples, - quotaWindowSizeSeconds: Int = - ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds) - -object ClientQuotaMetricsConfig { - val QuotaBytesPerSecondDefault = Long.MaxValue - val QuotaBytesPerSecondOverrides = "" - val DefaultQuotaDelayFactor = 1 - // Always have 10 whole windows + 1 current window - val DefaultNumQuotaSamples = 11 - val DefaultQuotaWindowSizeSeconds = 1 - val MaxThrottleTimeSeconds = 30 -} - -/** - * Helper class that records per-client metrics. It is also responsible for maintaining Quota usage statistics - * for all clients. - * @param config @ClientQuotaMetricsConfig quota configs - * @param metrics @Metrics Metrics instance - * @param apiKey API Key for the request - * @param time @Time object to use - */ -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.quotaBytesPerSecondOverrides) - private val defaultQuota = Quota.lessThan(config.quotaBytesPerSecondDefault) - private val lock = new ReentrantReadWriteLock() - private val delayQueue = new DelayQueue[ThrottledRequest]() - val throttledRequestReaper = new ThrottledRequestReaper(delayQueue) - throttledRequestReaper.start() - - private val delayQueueSensor = metrics.sensor(apiKey + "-delayQueue") - delayQueueSensor.add(new MetricName("queue-size", - apiKey, - "Tracks the size of the delay queue"), new Total()) - - /** - * 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) { - // Decrement the size of the delay queue - delayQueueSensor.record(-1) - trace("Response throttled for: " + response.delayTimeMs + " ms") - response.execute() - } - } - } - - /** - * Records that a clientId changed some metric being throttled (produced/consumed bytes, QPS etc.) - * @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 recordAndMaybeThrottle(clientId: String, value: Int, callback: => Unit) : Int = { - val clientSensors = getOrCreateQuotaSensors(clientId) - var delayTime = 0.0 - try { - clientSensors.quotaSensor.record(value) - // trigger the callback immediately if quota is not violated - callback - } catch { - case qve: QuotaViolationException => - delayTime = qve.getDelayTimeMs * config.quotaDelayFactor - delayQueue.add(new ThrottledRequest(time, delayTime.toLong, callback)) - delayQueueSensor.record() - clientSensors.throttleTimeSensor.record(delayTime) - // If delayed, add the element to the delayQueue - logger.debug("Quota violated for sensor (%s). Delay time: (%f)".format(clientSensors.quotaSensor.name(), delayTime), qve) - } - delayTime.toInt - } - - /** - * Returns the consumer quota for the specified clientId - * @return - */ - private[server] def quota(clientId: String) : Quota = overriddenQuota.getOrElse(clientId, 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) : ClientSensors = { - - // 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", - apiKey, - "Tracking throttle-time per client", - "client-id", - clientId), new Rate()) - 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 - ClientSensors(quotaSensor, throttleTimeSensor) - } - - private def getQuotaMetricConfig(quota: Quota) : MetricConfig = { - new MetricConfig() - .timeWindow(config.quotaWindowSizeSeconds, TimeUnit.SECONDS) - .samples(config.numQuotaSamples) - .quota(quota) - } - - /* Construct a Map of (clientId -> Quota) - * The input config is specified as a comma-separated K=V pairs - */ - private def initQuotaMap(input: String) : Map[String, Quota] = { - // If empty input, return an empty map - if (input.trim.length == 0) - Map[String, Quota]() - else - input.split(",").map(entry => { - val trimmedEntry = entry.trim - 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)) - pair(0) -> new Quota(pair(1).toDouble, true) - }).toMap - } - - def shutdown() = { - throttledRequestReaper.shutdown() - } -} \ 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 70fe9db..5bc30e3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -290,8 +290,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - val requestKey = RequestKeys.nameForKey(RequestKeys.ProduceKey) - quotaManagers.get(requestKey) match { + quotaManagers.get(RequestKeys.ProduceKey) match { case Some(quotaManager) => quotaManager.recordAndMaybeThrottle(produceRequest.clientId, numBytesAppended, produceResponseCallback) case None => @@ -347,12 +346,11 @@ class KafkaApis(val requestChannel: RequestChannel, if (fetchRequest.isFromFollower) { fetchResponseCallback } else { - val requestKey = RequestKeys.nameForKey(RequestKeys.FetchKey) - quotaManagers.get(requestKey) match { + quotaManagers.get(RequestKeys.FetchKey) match { case Some(quotaManager) => quotaManager.recordAndMaybeThrottle(fetchRequest.clientId, response.sizeInBytes, fetchResponseCallback) case None => - warn("Cannot throttle Api key %s".format(requestKey)) + warn("Cannot throttle Api key %s".format(RequestKeys.nameForKey(RequestKeys.FetchKey))) } } } @@ -638,8 +636,8 @@ class KafkaApis(val requestChannel: RequestChannel, /* * 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( + private def instantiateQuotaManagers(cfg: KafkaConfig): Map[Short, ClientQuotaManager] = { + val producerQuotaManagerCfg = ClientQuotaManagerConfig( quotaBytesPerSecondDefault = cfg.producerQuotaBytesPerSecondDefault, quotaBytesPerSecondOverrides = cfg.producerQuotaBytesPerSecondOverrides, quotaDelayFactor = cfg.quotaDelayFactor, @@ -647,7 +645,7 @@ class KafkaApis(val requestChannel: RequestChannel, quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds ) - val consumerQuotaManagerCfg = ClientQuotaMetricsConfig( + val consumerQuotaManagerCfg = ClientQuotaManagerConfig( quotaBytesPerSecondDefault = cfg.consumerQuotaBytesPerSecondDefault, quotaBytesPerSecondOverrides = cfg.consumerQuotaBytesPerSecondOverrides, quotaDelayFactor = cfg.quotaDelayFactor, @@ -655,17 +653,19 @@ class KafkaApis(val requestChannel: RequestChannel, quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds ) - val quotaManagers = Map[String, ClientQuotaMetrics]( - 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 org.apache.kafka.common.utils.SystemTime) + val quotaManagers = Map[Short, ClientQuotaManager]( + RequestKeys.ProduceKey -> + new ClientQuotaManager(producerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.ProduceKey), new org.apache.kafka.common.utils.SystemTime), + RequestKeys.FetchKey -> + new ClientQuotaManager(consumerQuotaManagerCfg, metrics, RequestKeys.nameForKey(RequestKeys.FetchKey), new org.apache.kafka.common.utils.SystemTime) ) quotaManagers } def close() { - quotaManagers.foreach(entry => entry._2.shutdown()) - info("Shut down complete.") + quotaManagers.foreach { case(apiKey, quotaManager) => + quotaManager.shutdown() + } + info("Shutdown complete.") } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 95d3df2..1cef3e7 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -136,13 +136,13 @@ object Defaults { val OffsetCommitRequiredAcks = OffsetManagerConfig.DefaultOffsetCommitRequiredAcks /** ********* Quota Configuration ***********/ - val ProducerQuotaBytesPerSecondDefault = ClientQuotaMetricsConfig.QuotaBytesPerSecondDefault - val ConsumerQuotaBytesPerSecondDefault = ClientQuotaMetricsConfig.QuotaBytesPerSecondDefault - val ProducerQuotaBytesPerSecondOverrides = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides - val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaMetricsConfig.QuotaBytesPerSecondOverrides - val QuotaDelayFactor : Double = ClientQuotaMetricsConfig.DefaultQuotaDelayFactor - val NumQuotaSamples : Int = ClientQuotaMetricsConfig.DefaultNumQuotaSamples - val QuotaWindowSizeSeconds : Int = ClientQuotaMetricsConfig.DefaultQuotaWindowSizeSeconds + val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault + val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault + val ProducerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides + val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides + val QuotaDelayFactor : Double = ClientQuotaManagerConfig.DefaultQuotaDelayFactor + val NumQuotaSamples : Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples + val QuotaWindowSizeSeconds : Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds val DeleteTopicEnable = false diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index c92369d..6d65507 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -358,7 +358,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg CoreUtils.swallow(kafkaController.shutdown()) if(zkClient != null) CoreUtils.swallow(zkClient.close()) - if(metrics != null) + if (metrics != null) CoreUtils.swallow(metrics.close()) brokerState.newState(NotRunning) diff --git a/core/src/main/scala/kafka/server/ThrottledRequest.scala b/core/src/main/scala/kafka/server/ThrottledRequest.scala deleted file mode 100644 index 103a951..0000000 --- a/core/src/main/scala/kafka/server/ThrottledRequest.scala +++ /dev/null @@ -1,46 +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, Delayed} - -import org.apache.kafka.common.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 - - def execute() = callback - - override def getDelay(unit: TimeUnit): Long = { - unit.convert(endTime - time.milliseconds, TimeUnit.MILLISECONDS) - } - - override 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/server/ThrottledResponse.scala b/core/src/main/scala/kafka/server/ThrottledResponse.scala new file mode 100644 index 0000000..1f80d54 --- /dev/null +++ b/core/src/main/scala/kafka/server/ThrottledResponse.scala @@ -0,0 +1,46 @@ +/** + * 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 org.apache.kafka.common.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 ThrottledResponse(val time: Time, val delayTimeMs: Long, callback: => Unit) extends Delayed { + val endTime = time.milliseconds + delayTimeMs + + def execute() = callback + + override def getDelay(unit: TimeUnit): Long = { + unit.convert(endTime - time.milliseconds, TimeUnit.MILLISECONDS) + } + + override def compareTo(d: Delayed): Int = { + val other = d.asInstanceOf[ThrottledResponse] + if (this.endTime < other.endTime) -1 + else if (this.endTime > other.endTime) 1 + else 0 + } +} diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index 44daa9a..d6c546c 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -20,7 +20,7 @@ import junit.framework.Assert import kafka.consumer.SimpleConsumer import kafka.integration.KafkaServerTestHarness import kafka.server.{KafkaServer, KafkaConfig} -import kafka.utils.{ZkUtils, TestUtils} +import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer._ import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback @@ -125,67 +125,47 @@ class QuotasTest extends KafkaServerTestHarness { val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala val numRecords = 1000 - val startTime = System.currentTimeMillis() - val bytesProduced = produce(producers.head, numRecords) - val elapsedTime = System.currentTimeMillis() - startTime - - 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 1 metric - Assert.assertEquals(1, numAsserts) + produce(producers.head, numRecords) + + val producerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.ProduceKey), + "Tracking throttle-time per client", + "client-id", producerId1) + Assert.assertTrue("Should have been throttled", allMetrics(producerMetricName).value() > 0) // Consumer should read in a bursty manner and get throttled immediately consume(consumers.head, numRecords) // The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build() - val fetchResponse: FetchResponse = replicaConsumers.head.fetch(request) - // fetchResponse.throttleTime should be 0 - // Assert.assertEquals("Throttle Time should be 0", 0, fetchResponse.throttleTime) - - for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { - if (name.tags().containsValue(consumerId1) && name.name().startsWith("throttle-time")) { - Assert.assertTrue("Should have been throttled", metric.value() > 0) - numAsserts += 1 - } - } + replicaConsumers.head.fetch(request) + val consumerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.FetchKey), + "Tracking throttle-time per client", + "client-id", consumerId1) + Assert.assertTrue("Should have been throttled", allMetrics(consumerMetricName).value() > 0) } @Test - def testProducerOverrideUnthrottled() { + def testProducerConsumerOverrideUnthrottled() { val allMetrics: mutable.Map[MetricName, KafkaMetric] = leaderNode.metrics.metrics().asScala val numRecords = 1000 - produce(producers(1), numRecords) - - var numAsserts = 0 - for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { - if (name.tags().containsValue(producerId2) && name.name().startsWith("throttle-time")) { - Assert.assertEquals("Should not have been throttled", 0.0, metric.value()) - numAsserts += 1 - } - } - // Should have matched 1 metric - Assert.assertEquals(1, numAsserts) + val producerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.ProduceKey), + "Tracking throttle-time per client", + "client-id", producerId2) + Assert.assertEquals("Should not have been throttled", 0.0, allMetrics(producerMetricName).value()) // The "client" consumer does not get throttled. - consume(consumers.head, numRecords) + consume(consumers(1), numRecords) // The replica consumer should not be throttled also. Create a fetch request which will exceed the quota immediately val request = new FetchRequestBuilder().addFetch(topic1, 0, 0, 1024*1024).replicaId(followerNode.config.brokerId).build() - val fetchResponse: FetchResponse = replicaConsumers(1).fetch(request) - // fetchResponse.throttleTime should be 0 - // Assert.assertEquals("Throttle Time should be 0", 0, fetchResponse.throttleTime) - - for ((name: MetricName, metric: KafkaMetric) <- allMetrics) { - if (name.tags().containsValue(consumerId2) && name.name().startsWith("throttle-time")) { - Assert.assertEquals("Should not have been throttled", 0.0, metric.value()) - numAsserts += 1 - } - } + replicaConsumers(1).fetch(request) + val consumerMetricName = new MetricName("throttle-time", + RequestKeys.nameForKey(RequestKeys.FetchKey), + "Tracking throttle-time per client", + "client-id", consumerId2) + Assert.assertEquals("Should not have been throttled", 0.0, allMetrics(consumerMetricName).value()) } def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): Int = { diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala new file mode 100644 index 0000000..831b5b8 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -0,0 +1,159 @@ +/** + * 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.api.RequestKeys +import org.apache.kafka.common.MetricName +import org.apache.kafka.common.metrics.{Metrics, Quota, MetricConfig} +import org.apache.kafka.common.utils.MockTime +import org.scalatest.junit.JUnit3Suite +import org.junit.{Before, Test, Assert} + +class ClientQuotaManagerTest extends JUnit3Suite { + private val time = new MockTime + + private val config = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000") + + var numCallbacks : Int = 0 + def callback { + numCallbacks += 1 + } + + @Before + def beforeMethod() { + numCallbacks = 0 + } + + @Test + def testQuotaParsing() { + val clientMetrics = new ClientQuotaManager(config, newMetrics, "producer", time) + try { + 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")) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testQuotaViolation() { + val metrics = newMetrics + val clientMetrics = new ClientQuotaManager(config, metrics, "producer", time) + val queueSizeMetric = metrics.metrics().get(new MetricName("queue-size", "producer", "")) + try { + /* 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.recordAndMaybeThrottle("unknown", 400, callback) + time.sleep(1000) + } + Assert.assertEquals(10, numCallbacks) + Assert.assertEquals(0, queueSizeMetric.value().toInt) + + // Create a spike. + // 400*10 + 2000 = 6000/10 = 600 bytes per second. + // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200 + val sleepTime = clientMetrics.recordAndMaybeThrottle("unknown", 2000, callback) + Assert.assertEquals("Should be throttled", 2200, sleepTime) + Assert.assertEquals(1, queueSizeMetric.value().toInt) + // After a request is delayed, the callback cannot be triggered immediately + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(10, numCallbacks) + time.sleep(sleepTime) + + // Callback can only be triggered after the the delay time passes + clientMetrics.throttledRequestReaper.doWork() + Assert.assertEquals(0, queueSizeMetric.value().toInt) + Assert.assertEquals(11, numCallbacks) + + // Could continue to see delays until the bursty sample disappears + for (i <- 0 until 10) { + clientMetrics.recordAndMaybeThrottle("unknown", 400, callback) + time.sleep(1000) + } + + Assert.assertEquals("Should be unthrottled since bursty sample has rolled over", + 0, clientMetrics.recordAndMaybeThrottle("unknown", 0, callback)) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testOverrideParse() { + var testConfig = ClientQuotaManagerConfig() + var clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + + try { + // Case 1 - Default config + Assert.assertEquals(new Quota(ClientQuotaManagerConfig.QuotaBytesPerSecondDefault, true), + clientMetrics.quota("p1")) + } finally { + clientMetrics.shutdown() + } + + + // Case 2 - Empty override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,,") + + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + try { + Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) + Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) + } finally { + clientMetrics.shutdown() + } + + // Case 3 - NumberFormatException for override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000,p2=4000,p3=p4") + try { + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "consumer", time) + Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) + } + catch { + // Swallow. + case nfe : NumberFormatException => + } + + // Case 4 - IllegalArgumentException for override + testConfig = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, + quotaBytesPerSecondOverrides = "p1=2000=3000") + try { + clientMetrics = new ClientQuotaManager(testConfig, newMetrics, "producer", time) + Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) + } + catch { + // Swallow. + case nfe : IllegalArgumentException => + } + + } + + def newMetrics: Metrics = { + new Metrics(new MetricConfig(), Collections.emptyList(), time) + } +} diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala deleted file mode 100644 index d44f682..0000000 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaMetricsTest.scala +++ /dev/null @@ -1,159 +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 kafka.api.RequestKeys -import org.apache.kafka.common.MetricName -import org.apache.kafka.common.metrics.{Metrics, Quota, MetricConfig} -import org.apache.kafka.common.utils.MockTime -import org.scalatest.junit.JUnit3Suite -import org.junit.{Before, Test, Assert} - -class ClientQuotaMetricsTest extends JUnit3Suite { - private val time = new MockTime - - private val config = ClientQuotaMetricsConfig(quotaBytesPerSecondDefault = 500, - quotaBytesPerSecondOverrides = "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, newMetrics, "producer", time) - try { - 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")) - } finally { - clientMetrics.shutdown() - } - } - - @Test - def testQuotaViolation() { - val metrics = newMetrics - val clientMetrics = new ClientQuotaMetrics(config, metrics, "producer", time) - val queueSizeMetric = metrics.metrics().get(new MetricName("queue-size", "producer", "")) - try { - /* 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.recordAndMaybeThrottle("unknown", 400, callback) - time.sleep(1000) - } - Assert.assertEquals(10, numCallbacks) - Assert.assertEquals(0, queueSizeMetric.value().toInt) - - // Create a spike. - // 400*10 + 2000 = 6000/10 = 600 bytes per second. - // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200 - val sleepTime = clientMetrics.recordAndMaybeThrottle("unknown", 2000, callback) - Assert.assertEquals("Should be throttled", 2200, sleepTime) - Assert.assertEquals(1, queueSizeMetric.value().toInt) - // After a request is delayed, the callback cannot be triggered immediately - clientMetrics.throttledRequestReaper.doWork() - Assert.assertEquals(10, numCallbacks) - time.sleep(sleepTime) - - // Callback can only be triggered after the the delay time passes - clientMetrics.throttledRequestReaper.doWork() - Assert.assertEquals(0, queueSizeMetric.value().toInt) - Assert.assertEquals(11, numCallbacks) - - // Could continue to see delays until the bursty sample disappears - for(i <- 0 until 10) { - clientMetrics.recordAndMaybeThrottle("unknown", 400, callback) - time.sleep(1000) - } - - Assert.assertEquals("Should be unthrottled since bursty sample has rolled over", - 0, clientMetrics.recordAndMaybeThrottle("unknown", 0, callback)) - } finally { - clientMetrics.shutdown() - } - } - - @Test - def testOverrideParse() { - var testConfig = ClientQuotaMetricsConfig() - var clientMetrics = new ClientQuotaMetrics(testConfig, newMetrics, "consumer", time) - - try { - // Case 1 - Default config - Assert.assertEquals(new Quota(ClientQuotaMetricsConfig.QuotaBytesPerSecondDefault, true), - clientMetrics.quota("p1")) - } finally { - clientMetrics.shutdown() - } - - - // Case 2 - Empty override - testConfig = ClientQuotaMetricsConfig(quotaBytesPerSecondDefault = 500, - quotaBytesPerSecondOverrides = "p1=2000,p2=4000,,") - - clientMetrics = new ClientQuotaMetrics(testConfig, newMetrics, "consumer", time) - try { - Assert.assertEquals(new Quota(2000, true), clientMetrics.quota("p1")) - Assert.assertEquals(new Quota(4000, true), clientMetrics.quota("p2")) - } finally { - clientMetrics.shutdown() - } - - // Case 3 - NumberFormatException for override - testConfig = ClientQuotaMetricsConfig(quotaBytesPerSecondDefault = 500, - quotaBytesPerSecondOverrides = "p1=2000,p2=4000,p3=p4") - try { - clientMetrics = new ClientQuotaMetrics(testConfig, newMetrics, "consumer", time) - Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) - } - catch { - // Swallow. - case nfe : NumberFormatException => - } - - // Case 4 - IllegalArgumentException for override - testConfig = ClientQuotaMetricsConfig(quotaBytesPerSecondDefault = 500, - quotaBytesPerSecondOverrides = "p1=2000=3000") - try { - clientMetrics = new ClientQuotaMetrics(testConfig, newMetrics, "producer", time) - Assert.fail("Should fail to parse invalid config " + testConfig.quotaBytesPerSecondOverrides) - } - catch { - // Swallow. - case nfe : IllegalArgumentException => - } - - } - - def newMetrics: Metrics = { - new Metrics(new MetricConfig(), Collections.emptyList(), time) - } -} diff --git a/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala deleted file mode 100644 index 1c88997..0000000 --- a/core/src/test/scala/unit/kafka/server/ThrottledRequestExpirationTest.scala +++ /dev/null @@ -1,90 +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 java.util.concurrent.{TimeUnit, DelayQueue} - -import org.apache.kafka.common.metrics.MetricConfig -import org.apache.kafka.common.utils.MockTime -import org.junit.{AfterClass, Before, Assert, Test} -import org.scalatest.junit.JUnit3Suite - -class ThrottledRequestExpirationTest extends JUnit3Suite { - 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) - - def callback { - numCallbacks += 1 - } - - @Before - def beforeMethod() { - numCallbacks = 0 - } - - @Test - def testExpire() { - val clientMetrics = new ClientQuotaMetrics(ClientQuotaMetricsConfig(), metrics, "producer", time) - - val delayQueue = new DelayQueue[ThrottledRequest]() - val reaper = new clientMetrics.ThrottledRequestReaper(delayQueue) - try { - // 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) - } finally { - clientMetrics.shutdown() - } - } - - @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) - } - } -} diff --git a/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala new file mode 100644 index 0000000..f4dd468 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala @@ -0,0 +1,90 @@ +/** + * 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 java.util.concurrent.{TimeUnit, DelayQueue} + +import org.apache.kafka.common.metrics.MetricConfig +import org.apache.kafka.common.utils.MockTime +import org.junit.{AfterClass, Before, Assert, Test} +import org.scalatest.junit.JUnit3Suite + +class ThrottledResponseExpirationTest extends JUnit3Suite { + 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) + + def callback { + numCallbacks += 1 + } + + @Before + def beforeMethod() { + numCallbacks = 0 + } + + @Test + def testExpire() { + val clientMetrics = new ClientQuotaManager(ClientQuotaManagerConfig(), metrics, "producer", time) + + val delayQueue = new DelayQueue[ThrottledResponse]() + val reaper = new clientMetrics.ThrottledRequestReaper(delayQueue) + try { + // Add 4 elements to the queue out of order. Add 2 elements with the same expire timestamp + delayQueue.add(new ThrottledResponse(time, 10, callback)) + delayQueue.add(new ThrottledResponse(time, 30, callback)) + delayQueue.add(new ThrottledResponse(time, 30, callback)) + delayQueue.add(new ThrottledResponse(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) + } finally { + clientMetrics.shutdown() + } + } + + @Test + def testThrottledRequest() { + val t1 : ThrottledResponse = new ThrottledResponse(time, 10, callback) + val t2 : ThrottledResponse = new ThrottledResponse(time, 20, callback) + val t3 : ThrottledResponse = new ThrottledResponse(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 81f6113f2e8c986442b46b8a6572473c172fb205 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 4 Aug 2015 19:07:24 -0700 Subject: [PATCH 10/21] Addressing comments --- .../src/main/java/org/apache/kafka/common/metrics/Quota.java | 10 ++-------- core/src/main/scala/kafka/server/KafkaConfig.scala | 10 +++++----- .../test/scala/unit/kafka/server/ClientQuotaManagerTest.scala | 6 +++--- .../unit/kafka/server/ThrottledResponseExpirationTest.scala | 8 ++++---- 4 files changed, 14 insertions(+), 20 deletions(-) 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 e87edb5..a3535dc 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 @@ -62,15 +62,9 @@ public final class Quota { public boolean equals(Object obj) { if (this == obj) return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) + if (!(obj instanceof Quota)) return false; Quota that = (Quota) obj; - if (that.bound != this.bound) - return false; - if (that.upper != this.upper) - return false; - return true; + return (that.bound == this.bound) && (this.upper == this.upper); } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 1cef3e7..e9b8393 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -140,9 +140,9 @@ object Defaults { val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault val ProducerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides - val QuotaDelayFactor : Double = ClientQuotaManagerConfig.DefaultQuotaDelayFactor - val NumQuotaSamples : Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples - val QuotaWindowSizeSeconds : Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds + val QuotaDelayFactor: Double = ClientQuotaManagerConfig.DefaultQuotaDelayFactor + val NumQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples + val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds val DeleteTopicEnable = false @@ -705,8 +705,8 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) /** ********* Quota Configuration **************/ - val producerQuotaDefaultBytesPerSecond = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp) - val consumerQuotaDefaultBytesPerSecond = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp) + val producerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp) + val consumerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp) val producerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp) val consumerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp) val quotaDelayFactor = getDouble(KafkaConfig.QuotaDelayFactorProp) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 831b5b8..97dcca8 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -31,7 +31,7 @@ class ClientQuotaManagerTest extends JUnit3Suite { private val config = ClientQuotaManagerConfig(quotaBytesPerSecondDefault = 500, quotaBytesPerSecondOverrides = "p1=2000,p2=4000") - var numCallbacks : Int = 0 + var numCallbacks: Int = 0 def callback { numCallbacks += 1 } @@ -136,7 +136,7 @@ class ClientQuotaManagerTest extends JUnit3Suite { } catch { // Swallow. - case nfe : NumberFormatException => + case nfe: NumberFormatException => } // Case 4 - IllegalArgumentException for override @@ -148,7 +148,7 @@ class ClientQuotaManagerTest extends JUnit3Suite { } catch { // Swallow. - case nfe : IllegalArgumentException => + case nfe: IllegalArgumentException => } } diff --git a/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala index f4dd468..14a7f45 100644 --- a/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ThrottledResponseExpirationTest.scala @@ -28,7 +28,7 @@ import org.scalatest.junit.JUnit3Suite class ThrottledResponseExpirationTest extends JUnit3Suite { private val time = new MockTime - private var numCallbacks : Int = 0 + private var numCallbacks: Int = 0 private val metrics = new org.apache.kafka.common.metrics.Metrics(new MetricConfig(), Collections.emptyList(), time) @@ -73,9 +73,9 @@ class ThrottledResponseExpirationTest extends JUnit3Suite { @Test def testThrottledRequest() { - val t1 : ThrottledResponse = new ThrottledResponse(time, 10, callback) - val t2 : ThrottledResponse = new ThrottledResponse(time, 20, callback) - val t3 : ThrottledResponse = new ThrottledResponse(time, 20, callback) + val t1: ThrottledResponse = new ThrottledResponse(time, 10, callback) + val t2: ThrottledResponse = new ThrottledResponse(time, 20, callback) + val t3: ThrottledResponse = new ThrottledResponse(time, 20, callback) Assert.assertEquals(10, t1.delayTimeMs) Assert.assertEquals(20, t2.delayTimeMs) Assert.assertEquals(20, t3.delayTimeMs) -- 1.7.12.4 From 98ebc9c126c2556d0356b62836353986b48e9cb8 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Fri, 7 Aug 2015 11:27:41 -0700 Subject: [PATCH 11/21] Addressing Juns comments --- clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java | 3 ++- core/src/main/scala/kafka/server/ClientQuotaManager.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 7 +------ core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- 4 files changed, 5 insertions(+), 9 deletions(-) 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 901741f..8c36afe 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 @@ -130,7 +130,8 @@ public final class Sensor { /* * This calculates the amount of time needed to bring the metric within quota - * assuming that no new metrics are recorded + * assuming that no new metrics are recorded. + * This calculation only makes sense for Rate metrics */ private int delayTime(Quota quota, double metricValue, MetricConfig config) { double difference = metricValue - quota.bound(); diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 66a66f5..7f13e80 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -128,7 +128,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, delayQueueSensor.record() clientSensors.throttleTimeSensor.record(delayTime) // If delayed, add the element to the delayQueue - logger.debug("Quota violated for sensor (%s). Delay time: (%f)".format(clientSensors.quotaSensor.name(), delayTime), qve) + logger.debug("Quota violated for sensor (%s). Delay time: (%f)".format(clientSensors.quotaSensor.name(), delayTime)) } delayTime.toInt } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5bc30e3..6381d06 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -290,12 +290,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - quotaManagers.get(RequestKeys.ProduceKey) match { - case Some(quotaManager) => - quotaManager.recordAndMaybeThrottle(produceRequest.clientId, numBytesAppended, produceResponseCallback) - case None => - warn("Cannot throttle producer request for clientId: %s".format(produceRequest.clientId)) - } + quotaManagers(RequestKeys.ProduceKey).recordAndMaybeThrottle(produceRequest.clientId, numBytesAppended, produceResponseCallback) } // only allow appending to internal topic partitions diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index e9b8393..d78f39d 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -416,7 +416,7 @@ object KafkaConfig { "Example: clientIdX=10485760,clientIdY=10485760" 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 QuotaDelayFactorDoc = "The quota delay factor modifies any delay by a fixed multiplier (default 1)" val NumQuotaSamplesDoc = "The number of samples to retain in memory" val QuotaWindowSizeSecondsDoc = "The time span of each sample" -- 1.7.12.4 From 8eea5d7e7439032d1d3be892fd9b9d0da9453415 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 10 Aug 2015 11:01:18 -0700 Subject: [PATCH 12/21] Minor checkstyle changes --- .../java/org/apache/kafka/common/metrics/QuotaViolationException.java | 4 +++- clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java | 2 +- .../src/test/java/org/apache/kafka/common/metrics/MetricsTest.java | 4 ++-- 3 files changed, 6 insertions(+), 4 deletions(-) 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 742fc14..7265687 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 @@ -31,5 +31,7 @@ public class QuotaViolationException extends KafkaException { 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 8c36afe..923f8dc 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 @@ -131,7 +131,7 @@ public final class Sensor { /* * This calculates the amount of time needed to bring the metric within quota * assuming that no new metrics are recorded. - * This calculation only makes sense for Rate metrics + * This delay calculation is only applicable to Rate metrics */ private int delayTime(Quota quota, double metricValue, MetricConfig config) { double difference = metricValue - quota.bound(); diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index d5d0281..0a7dcd8 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -85,7 +85,7 @@ public class MetricsTest { } // prior to any time passing double elapsedSecs = (config.timeWindowMs() * (config.samples() - 1)) / 1000.0; - assertEquals(String.format("Occurrences(0...%d) = %d", count, count/elapsedSecs), count / elapsedSecs, + assertEquals(String.format("Occurrences(0...%d) = %f", count, count / elapsedSecs), count / elapsedSecs, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); // pretend 2 seconds passed... @@ -99,7 +99,7 @@ public class MetricsTest { assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get(new MetricName("test.min", "grp1")).value(), EPS); assertEquals("Rate(0...9) = 1.40625", sum / elapsedSecs, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); - assertEquals(String.format("Occurrences(0...%d) = %d", count, count/elapsedSecs), + assertEquals(String.format("Occurrences(0...%d) = %f", count, count / elapsedSecs), count / elapsedSecs, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); assertEquals("Count(0...9) = 10", -- 1.7.12.4 From e3c30d05ab2d7c705749287245b83e120d599501 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 10 Aug 2015 13:48:21 -0700 Subject: [PATCH 13/21] fixed test case --- core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 8ee7f9e..02144d0 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -474,8 +474,8 @@ class KafkaConfigTest { 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.ProducerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") + case KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp => // ignore string case KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp => // ignore string case KafkaConfig.QuotaDelayFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") -- 1.7.12.4 From bdb4512458e740f7dfc4374a57ac6e0ffe69499c Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Mon, 10 Aug 2015 21:57:36 -0700 Subject: [PATCH 14/21] Addressing Juns comments --- .../java/org/apache/kafka/common/metrics/Sensor.java | 19 +++++++++++++++++-- .../main/scala/kafka/server/ClientQuotaManager.scala | 8 ++++---- .../test/scala/integration/kafka/api/QuotasTest.scala | 4 ++-- 3 files changed, 23 insertions(+), 8 deletions(-) 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 923f8dc..d847b47 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 @@ -130,8 +130,23 @@ public final class Sensor { /* * This calculates the amount of time needed to bring the metric within quota - * assuming that no new metrics are recorded. - * This delay calculation is only applicable to Rate metrics + * assuming that no new metrics are recorded. + * This delay calculation is only applicable to Rate metrics. + * + * The formula for computing the delay time is: ((currentRateValue - quotaRate)/quotaRate) * numWindows * windowSize + * This is not completely accurate because the final window may be incomplete. + * TODO: We should use elapsed time of the final sample. + * + * Lets take an example (all in seconds): + * quotaRate = 10QPS + * elapedSec = 20 (Lets say 20 windows of 1 second each. The last second is partially complete but we take the entire second) + * currentValueOfMetric = 250 + * currentRate = (250/20) = 12.5 (assuming 20 elapsed seconds. Current second may be incomplete) + * + * Current Sensor delay = ((currentRateValue - quotaRate)/quotaRate) * numWindows * windowSize + * = ((12.5 - 10)/10) * 20 windows * 1 second window = 2.5/10 * 20 = 20/4 + * = 5 second delay + * */ private int delayTime(Quota quota, double metricValue, MetricConfig config) { double difference = metricValue - quota.bound(); diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 7f13e80..5ca6922 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -21,7 +21,7 @@ import java.util.concurrent.{DelayQueue, TimeUnit} import kafka.utils.{ShutdownableThread, Logging} import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ -import org.apache.kafka.common.metrics.stats.{Total, Rate} +import org.apache.kafka.common.metrics.stats.{Total, Rate, Avg} import java.util.concurrent.locks.ReentrantReadWriteLock import org.apache.kafka.common.utils.Time @@ -186,9 +186,9 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, throttleTimeSensor = metrics.sensor(throttleTimeSensorName) throttleTimeSensor.add(new MetricName("throttle-time", apiKey, - "Tracking throttle-time per client", + "Tracking average throttle-time per client", "client-id", - clientId), new Rate()) + clientId), new Avg()) quotaSensor = metrics.sensor(quotaSensorName, getQuotaMetricConfig(quota(clientId))) quotaSensor.add(new MetricName("byte-rate", apiKey, @@ -231,4 +231,4 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, def shutdown() = { throttledRequestReaper.shutdown() } -} \ No newline at end of file +} diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index d6c546c..ba5c278 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -154,7 +154,7 @@ class QuotasTest extends KafkaServerTestHarness { RequestKeys.nameForKey(RequestKeys.ProduceKey), "Tracking throttle-time per client", "client-id", producerId2) - Assert.assertEquals("Should not have been throttled", 0.0, allMetrics(producerMetricName).value()) + Assert.assertEquals("Should not have been throttled", Double.NaN, allMetrics(producerMetricName).value()) // The "client" consumer does not get throttled. consume(consumers(1), numRecords) @@ -165,7 +165,7 @@ class QuotasTest extends KafkaServerTestHarness { RequestKeys.nameForKey(RequestKeys.FetchKey), "Tracking throttle-time per client", "client-id", consumerId2) - Assert.assertEquals("Should not have been throttled", 0.0, allMetrics(consumerMetricName).value()) + Assert.assertEquals("Should not have been throttled", Double.NaN, allMetrics(consumerMetricName).value()) } def produce(p: KafkaProducer[Array[Byte], Array[Byte]], count: Int): Int = { -- 1.7.12.4 From 6df18c1119629f820bce30db97f87514533a56db Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 12 Aug 2015 10:28:09 -0700 Subject: [PATCH 15/21] Addressing Juns comments --- .../java/org/apache/kafka/common/metrics/Sensor.java | 17 +++-------------- .../main/scala/kafka/server/ClientQuotaManager.scala | 7 +------ core/src/main/scala/kafka/server/KafkaApis.scala | 2 -- core/src/main/scala/kafka/server/KafkaConfig.scala | 5 ----- .../test/scala/unit/kafka/server/KafkaConfigTest.scala | 1 - 5 files changed, 4 insertions(+), 28 deletions(-) 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 d847b47..faab708 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 @@ -133,20 +133,9 @@ public final class Sensor { * assuming that no new metrics are recorded. * This delay calculation is only applicable to Rate metrics. * - * The formula for computing the delay time is: ((currentRateValue - quotaRate)/quotaRate) * numWindows * windowSize - * This is not completely accurate because the final window may be incomplete. - * TODO: We should use elapsed time of the final sample. - * - * Lets take an example (all in seconds): - * quotaRate = 10QPS - * elapedSec = 20 (Lets say 20 windows of 1 second each. The last second is partially complete but we take the entire second) - * currentValueOfMetric = 250 - * currentRate = (250/20) = 12.5 (assuming 20 elapsed seconds. Current second may be incomplete) - * - * Current Sensor delay = ((currentRateValue - quotaRate)/quotaRate) * numWindows * windowSize - * = ((12.5 - 10)/10) * 20 windows * 1 second window = 2.5/10 * 20 = 20/4 - * = 5 second delay - * + * Basically, if O is the observed rate and T is the target rate over a window of W, to bring O down to T, + * we need to add a delay of X to W such that O * W / (W + X) = T. + * Solving for X, we get X = (O - T)/T * W. */ private int delayTime(Quota quota, double metricValue, MetricConfig config) { double difference = metricValue - quota.bound(); diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 5ca6922..de358bb 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -37,7 +37,6 @@ private case class ClientSensors(quotaSensor: Sensor, throttleTimeSensor: Sensor * Configuration settings for quota management * @param quotaBytesPerSecondDefault 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 numQuotaSamples The number of samples to retain in memory * @param quotaWindowSizeSeconds The time span of each sample * @@ -46,8 +45,6 @@ case class ClientQuotaManagerConfig(quotaBytesPerSecondDefault: Long = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault, quotaBytesPerSecondOverrides: String = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides, - quotaDelayFactor: Double = - ClientQuotaManagerConfig.DefaultQuotaDelayFactor, numQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples, quotaWindowSizeSeconds: Int = @@ -56,7 +53,6 @@ case class ClientQuotaManagerConfig(quotaBytesPerSecondDefault: Long = object ClientQuotaManagerConfig { val QuotaBytesPerSecondDefault = Long.MaxValue val QuotaBytesPerSecondOverrides = "" - val DefaultQuotaDelayFactor = 1 // Always have 10 whole windows + 1 current window val DefaultNumQuotaSamples = 11 val DefaultQuotaWindowSizeSeconds = 1 @@ -123,8 +119,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, callback } catch { case qve: QuotaViolationException => - delayTime = qve.getDelayTimeMs * config.quotaDelayFactor - delayQueue.add(new ThrottledResponse(time, delayTime.toLong, callback)) + delayQueue.add(new ThrottledResponse(time, qve.getDelayTimeMs, callback)) delayQueueSensor.record() clientSensors.throttleTimeSensor.record(delayTime) // If delayed, add the element to the delayQueue diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6381d06..67f0cad 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -635,7 +635,6 @@ class KafkaApis(val requestChannel: RequestChannel, val producerQuotaManagerCfg = ClientQuotaManagerConfig( quotaBytesPerSecondDefault = cfg.producerQuotaBytesPerSecondDefault, quotaBytesPerSecondOverrides = cfg.producerQuotaBytesPerSecondOverrides, - quotaDelayFactor = cfg.quotaDelayFactor, numQuotaSamples = cfg.numQuotaSamples, quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds ) @@ -643,7 +642,6 @@ class KafkaApis(val requestChannel: RequestChannel, val consumerQuotaManagerCfg = ClientQuotaManagerConfig( quotaBytesPerSecondDefault = cfg.consumerQuotaBytesPerSecondDefault, quotaBytesPerSecondOverrides = cfg.consumerQuotaBytesPerSecondOverrides, - quotaDelayFactor = cfg.quotaDelayFactor, numQuotaSamples = cfg.numQuotaSamples, quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds ) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index d78f39d..394f21b 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -140,7 +140,6 @@ object Defaults { val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaBytesPerSecondDefault val ProducerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides val ConsumerQuotaBytesPerSecondOverrides = ClientQuotaManagerConfig.QuotaBytesPerSecondOverrides - val QuotaDelayFactor: Double = ClientQuotaManagerConfig.DefaultQuotaDelayFactor val NumQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds @@ -268,7 +267,6 @@ object KafkaConfig { val ConsumerQuotaBytesPerSecondDefaultProp = "quota.consumer.default" val ProducerQuotaBytesPerSecondOverridesProp = "quota.producer.bytes.per.second.overrides" val ConsumerQuotaBytesPerSecondOverridesProp = "quota.consumer.bytes.per.second.overrides" - val QuotaDelayFactorProp = "quota.delay.factor" val NumQuotaSamplesProp = "quota.window.num" val QuotaWindowSizeSecondsProp = "quota.window.size.seconds" @@ -416,7 +414,6 @@ object KafkaConfig { "Example: clientIdX=10485760,clientIdY=10485760" 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)" val NumQuotaSamplesDoc = "The number of samples to retain in memory" val QuotaWindowSizeSecondsDoc = "The time span of each sample" @@ -562,7 +559,6 @@ object KafkaConfig { .define(ConsumerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ConsumerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ConsumerQuotaBytesPerSecondDefaultDoc) .define(ProducerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ProducerQuotaBytesPerSecondOverrides, HIGH, ProducerQuotaBytesPerSecondOverridesDoc) .define(ConsumerQuotaBytesPerSecondOverridesProp, STRING, Defaults.ConsumerQuotaBytesPerSecondOverrides, HIGH, ConsumerQuotaBytesPerSecondOverridesDoc) - .define(QuotaDelayFactorProp, DOUBLE, Defaults.QuotaDelayFactor, atLeast(1), LOW, QuotaDelayFactorDoc) .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) } @@ -709,7 +705,6 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka val consumerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp) val producerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp) val consumerQuotaBytesPerSecondOverrides = getString(KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp) - val quotaDelayFactor = getDouble(KafkaConfig.QuotaDelayFactorProp) val numQuotaSamples = getInt(KafkaConfig.NumQuotaSamplesProp) val quotaWindowSizeSeconds = getInt(KafkaConfig.QuotaWindowSizeSecondsProp) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 02144d0..9688b8c 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -478,7 +478,6 @@ class KafkaConfigTest { case KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.ProducerQuotaBytesPerSecondOverridesProp => // ignore string case KafkaConfig.ConsumerQuotaBytesPerSecondOverridesProp => // ignore string - case KafkaConfig.QuotaDelayFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.NumQuotaSamplesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") case KafkaConfig.QuotaWindowSizeSecondsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0") -- 1.7.12.4 From 974b96bba9371916e31a2183923c33a82fcd15b4 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 12 Aug 2015 12:00:44 -0700 Subject: [PATCH 16/21] Addressing Juns comments --- .../common/metrics/QuotaViolationException.java | 4 +++ .../org/apache/kafka/common/metrics/Sensor.java | 18 +------------ .../apache/kafka/common/metrics/stats/Rate.java | 31 +++++++++++++++++++++- .../scala/kafka/server/ClientQuotaManager.scala | 1 + 4 files changed, 36 insertions(+), 18 deletions(-) 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 7265687..9e948a8 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 @@ -26,6 +26,10 @@ public class QuotaViolationException extends KafkaException { private static final long serialVersionUID = 1L; private final int delayTimeMs; + public QuotaViolationException(String m) { + this(m, 0); + } + public QuotaViolationException(String m, int delayTimeMs) { super(m); this.delayTimeMs = 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 faab708..0aa74b5 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 @@ -120,29 +120,13 @@ public final class Sensor { "(%s) violated quota. Actual: (%f), Threshold: (%f)", metric.metricName(), quota.bound(), - value), - delayTime(quota, value, config)); + value)); } } } } } - /* - * This calculates the amount of time needed to bring the metric within quota - * assuming that no new metrics are recorded. - * This delay calculation is only applicable to Rate metrics. - * - * Basically, if O is the observed rate and T is the target rate over a window of W, to bring O down to T, - * we need to add a delay of X to W such that O * W / (W + X) = T. - * Solving for X, we get X = (O - T)/T * W. - */ - private int delayTime(Quota quota, double metricValue, MetricConfig config) { - double difference = metricValue - quota.bound(); - double time = difference / quota.bound() * config.samples() * config.timeWindowMs(); - return (int) Math.round(time); - } - /** * Register a compound statistic with this sensor with no config override */ diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index 3ae9f7c..135dfdb 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -17,6 +17,9 @@ import java.util.concurrent.TimeUnit; import org.apache.kafka.common.metrics.MeasurableStat; import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Quota; +import org.apache.kafka.common.metrics.QuotaViolationException; + /** * The rate of the given quantity. By default this is the total observed over a set of samples from a sampled statistic @@ -61,9 +64,35 @@ public class Rate implements MeasurableStat { // the elapsed time is always N-1 complete windows plus whatever fraction of the final window is complete long elapsedCurrentWindowMs = now - stat.current(now).lastWindowMs; long elapsedPriorWindowsMs = config.timeWindowMs() * (config.samples() - 1); - return value / convert(elapsedCurrentWindowMs + elapsedPriorWindowsMs); + double observedRate = value / convert(elapsedCurrentWindowMs + elapsedPriorWindowsMs); + Quota quota = config.quota(); + if (quota != null && !quota.acceptable(observedRate)) { + throw new QuotaViolationException(String.format( + "Quota Violation. Actual: (%f), Threshold: (%f)", + quota.bound(), + value), + delayTime(quota, observedRate, config)); + } + return observedRate; + } + + /* + * This calculates the amount of time needed to bring the metric within quota + * assuming that no new metrics are recorded. + * This delay calculation is only applicable to Rate metrics. + * + * Basically, if O is the observed rate and T is the target rate over a window of W, to bring O down to T, + * we need to add a delay of X to W such that O * W / (W + X) = T. + * Solving for X, we get X = (O - T)/T * W. + */ + private int delayTime(Quota quota, double metricValue, MetricConfig config) { + double difference = metricValue - quota.bound(); + double time = difference / quota.bound() * config.samples() * config.timeWindowMs(); + return (int) Math.round(time); } + + private double convert(long timeMs) { switch (unit) { case NANOSECONDS: diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index de358bb..6dcadfe 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -119,6 +119,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, callback } catch { case qve: QuotaViolationException => + delayTime = qve.getDelayTimeMs delayQueue.add(new ThrottledResponse(time, qve.getDelayTimeMs, callback)) delayQueueSensor.record() clientSensors.throttleTimeSensor.record(delayTime) -- 1.7.12.4 From bc0529b1cd38d1f368395907226ea56cd42874d8 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 12 Aug 2015 12:04:42 -0700 Subject: [PATCH 17/21] Minor change --- clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java | 2 -- 1 file changed, 2 deletions(-) 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 0aa74b5..8f90284 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,7 +37,6 @@ 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(); @@ -48,7 +47,6 @@ public final class Sensor { this.stats = new ArrayList(); this.config = config; this.time = time; - this.creationTime = time.milliseconds(); checkForest(new HashSet()); } -- 1.7.12.4 From 4ba583712609802a58d16ad0d1624ea8f3795257 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 12 Aug 2015 12:08:09 -0700 Subject: [PATCH 18/21] Minor change --- clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index 135dfdb..e3c1033 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -79,7 +79,6 @@ public class Rate implements MeasurableStat { /* * This calculates the amount of time needed to bring the metric within quota * assuming that no new metrics are recorded. - * This delay calculation is only applicable to Rate metrics. * * Basically, if O is the observed rate and T is the target rate over a window of W, to bring O down to T, * we need to add a delay of X to W such that O * W / (W + X) = T. -- 1.7.12.4 From 824656d5e30bb475fe74b2f4359e89977b8dd056 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 12 Aug 2015 21:22:41 -0700 Subject: [PATCH 19/21] Moving MockTime back to test --- build.gradle | 5 ++- .../org/apache/kafka/common/utils/MockTime.java | 43 ---------------------- .../org/apache/kafka/common/utils/MockTime.java | 43 ++++++++++++++++++++++ 3 files changed, 46 insertions(+), 45 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/utils/MockTime.java create mode 100644 clients/src/test/java/org/apache/kafka/common/utils/MockTime.java diff --git a/build.gradle b/build.gradle index 1b67e62..0e57c11 100644 --- a/build.gradle +++ b/build.gradle @@ -243,6 +243,7 @@ project(':core') { testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5" + testCompile project(path: ':clients', configuration: 'archives') testRuntime "$slf4jlog4j" @@ -379,9 +380,9 @@ project(':clients') { testRuntime "$slf4jlog4j" } - task testJar(type: Jar) { - classifier = 'test' + task testJar(type: Jar, dependsOn: testClasses) { from sourceSets.test.output + classifier = 'archives' } test { 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 deleted file mode 100644 index eb7fcf0..0000000 --- a/clients/src/main/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/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java new file mode 100644 index 0000000..eb7fcf0 --- /dev/null +++ b/clients/src/test/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); + } + +} -- 1.7.12.4 From d3f372966b19b3c1679594612a0bf657643ca2df Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 13 Aug 2015 19:01:13 -0700 Subject: [PATCH 20/21] Addressing Joels comments --- build.gradle | 4 ++-- .../main/java/org/apache/kafka/common/metrics/stats/Rate.java | 9 +++++---- .../test/scala/unit/kafka/server/ClientQuotaManagerTest.scala | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/build.gradle b/build.gradle index 0e57c11..03b1503 100644 --- a/build.gradle +++ b/build.gradle @@ -380,9 +380,9 @@ project(':clients') { testRuntime "$slf4jlog4j" } - task testJar(type: Jar, dependsOn: testClasses) { + task testJar(type: Jar) { + classifier = 'test' from sourceSets.test.output - classifier = 'archives' } test { diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index e3c1033..87dd5a0 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -64,14 +64,15 @@ public class Rate implements MeasurableStat { // the elapsed time is always N-1 complete windows plus whatever fraction of the final window is complete long elapsedCurrentWindowMs = now - stat.current(now).lastWindowMs; long elapsedPriorWindowsMs = config.timeWindowMs() * (config.samples() - 1); - double observedRate = value / convert(elapsedCurrentWindowMs + elapsedPriorWindowsMs); + long totalElapsedTime = elapsedCurrentWindowMs + elapsedPriorWindowsMs; + double observedRate = value / convert(totalElapsedTime); Quota quota = config.quota(); if (quota != null && !quota.acceptable(observedRate)) { throw new QuotaViolationException(String.format( "Quota Violation. Actual: (%f), Threshold: (%f)", quota.bound(), value), - delayTime(quota, observedRate, config)); + delayTime(quota, observedRate, config, totalElapsedTime)); } return observedRate; } @@ -84,9 +85,9 @@ public class Rate implements MeasurableStat { * we need to add a delay of X to W such that O * W / (W + X) = T. * Solving for X, we get X = (O - T)/T * W. */ - private int delayTime(Quota quota, double metricValue, MetricConfig config) { + private int delayTime(Quota quota, double metricValue, MetricConfig config, long totalElapsedTime) { double difference = metricValue - quota.bound(); - double time = difference / quota.bound() * config.samples() * config.timeWindowMs(); + double time = difference / quota.bound() * totalElapsedTime; return (int) Math.round(time); } diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 97dcca8..68a64d3 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -76,7 +76,7 @@ class ClientQuotaManagerTest extends JUnit3Suite { // 400*10 + 2000 = 6000/10 = 600 bytes per second. // (600 - quota)/quota*window-size = (600-500)/500*11 seconds = 2200 val sleepTime = clientMetrics.recordAndMaybeThrottle("unknown", 2000, callback) - Assert.assertEquals("Should be throttled", 2200, sleepTime) + Assert.assertEquals("Should be throttled", 2000, sleepTime) Assert.assertEquals(1, queueSizeMetric.value().toInt) // After a request is delayed, the callback cannot be triggered immediately clientMetrics.throttledRequestReaper.doWork() -- 1.7.12.4 From 0b8e6580a784b7197071fd21886bae69b19ea05d Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Thu, 13 Aug 2015 19:18:59 -0700 Subject: [PATCH 21/21] Minor import change --- core/src/test/scala/integration/kafka/api/QuotasTest.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index ba5c278..a11bf90 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics.KafkaMetric import org.junit.Assert._ -import org.junit.Test +import org.junit.{After, Before, Test} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -66,6 +66,7 @@ class QuotasTest extends KafkaServerTestHarness { var followerNode: KafkaServer = null private val topic1 = "topic-1" + @Before override def setUp() { super.setUp() val producerProps = new Properties() @@ -113,6 +114,7 @@ class QuotasTest extends KafkaServerTestHarness { } + @After override def tearDown() { producers.foreach( _.close ) consumers.foreach( _.close ) -- 1.7.12.4