You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by gw...@apache.org on 2015/10/30 02:34:07 UTC

kafka git commit: KAFKA-2502; Documentation for quotas

Repository: kafka
Updated Branches:
  refs/heads/trunk e0ca8fdd7 -> 13803ae91


KAFKA-2502; Documentation for quotas

Followed the approach specified here: https://issues.apache.org/jira/browse/KAFKA-2502
I also made a minor fix to ConfigCommand to expose the right options on add-config.

Author: Aditya Auradkar <aa...@linkedin.com>

Reviewers: Gwen Shapira

Closes #381 from auradkar/K-2502


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/13803ae9
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/13803ae9
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/13803ae9

Branch: refs/heads/trunk
Commit: 13803ae91cc2d25ef42b71f3737e6c2ee7eb8809
Parents: e0ca8fd
Author: Aditya Auradkar <aa...@linkedin.com>
Authored: Thu Oct 29 18:33:56 2015 -0700
Committer: Gwen Shapira <cs...@gmail.com>
Committed: Thu Oct 29 18:33:56 2015 -0700

----------------------------------------------------------------------
 .../main/scala/kafka/admin/ConfigCommand.scala  |  5 +--
 docs/design.html                                | 25 ++++++++++++++
 docs/ops.html                                   | 35 ++++++++++++++++++++
 3 files changed, 63 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/13803ae9/core/src/main/scala/kafka/admin/ConfigCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala
index a6984be..3e9293c 100644
--- a/core/src/main/scala/kafka/admin/ConfigCommand.scala
+++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala
@@ -22,7 +22,7 @@ import java.util.Properties
 import kafka.admin.TopicCommand._
 import kafka.consumer.ConsumerConfig
 import kafka.log.{Defaults, LogConfig}
-import kafka.server.ConfigType
+import kafka.server.{ClientConfigOverride, ConfigType}
 import kafka.utils.{ZkUtils, CommandLineUtils}
 import org.I0Itec.zkclient.ZkClient
 import scala.collection._
@@ -150,7 +150,8 @@ object ConfigCommand {
     val nl = System.getProperty("line.separator")
     val addConfig = parser.accepts("add-config", "Key Value pairs configs to add 'k1=v1,k2=v2'. The following is a list of valid configurations: " +
             "For entity_type '" + ConfigType.Topic + "': " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl +
-            "For entity_type '" + ConfigType.Client + "' currently no configs are processed by the brokers")
+            "For entity_type '" + ConfigType.Client + "': " + nl + "\t" + ClientConfigOverride.ProducerOverride
+                                                            + nl + "\t" + ClientConfigOverride.ConsumerOverride)
             .withRequiredArg
             .ofType(classOf[String])
             .withValuesSeparatedBy(',')

http://git-wip-us.apache.org/repos/asf/kafka/blob/13803ae9/docs/design.html
----------------------------------------------------------------------
diff --git a/docs/design.html b/docs/design.html
index f0ab6ca..b7a6e7d 100644
--- a/docs/design.html
+++ b/docs/design.html
@@ -353,3 +353,28 @@ Further cleaner configurations are described <a href="/documentation.html#broker
   <li>You cannot configure yet how much log is retained without compaction (the "head" of the log).  Currently all segments are eligible except for the last segment, i.e. the one currently being written to.</li>
   <li>Log compaction is not yet compatible with compressed topics.</li>
 </ol>
+<h3><a id="semantics">4.9 Quotas</a></h3>
+<p>
+    Starting in 0.9, the Kafka cluster has the ability to enforce quotas on produce and fetch requests. Quotas are basically byte-rate thresholds defined per client-id. A client-id logically identifies an application making a request. Hence a single client-id can span multiple producer and consumer instances and the quota will apply for all of them as a single entity i.e. if client-id="test-client" has a produce quota of 10MB/sec, this is shared across all instances with that same id.
+
+<h4>Why are quotas necessary?</h4>
+<p>
+It is possible for producers and consumers to produce/consume very high volumes of data and thus monopolize broker resources, cause network saturation and generally DOS other clients and the brokers themselves. Having quotas protects against these issues and is all tbe more important in large multi-tenant clusters where a small set of badly behaved clients can degrade user experience for the well behaved ones. In fact, when running Kafka as a service this even makes it possible to enforce API limits according to an agreed upon contract.
+</p>
+<h4>Enforcement</h4>
+<p>
+    By default, each unique client-id receives a fixed quota in bytes/sec as configured by the cluster (quota.producer.default, quota.consumer.default).
+    This quota is defined on a per-broker basis. Each client can publish/fetch a maximum of X bytes/sec per broker before it gets throttled. We decided that defining these quotas per broker is much better than having a fixed cluster wide bandwidth per client because that would require a mechanism to share client quota usage among all the brokers. This can be harder to get right than the quota implementation itself!
+</p>
+<p>
+    How does a broker react when it detects a quota violation? In our solution, the broker does not return an error rather it attempts to slow down a client exceeding its quota. It computes the amount of delay needed to bring a guilty client under it's quota and delays the response for that time. This approach keeps the quota violation transparent to clients (outside of client side metrics). This also keeps them from having to implement any special backoff and retry behavior which can get tricky. In fact, bad client behavior (retry without backoff) can exacerbate the very problem quotas are trying to solve.
+</p>
+<p>
+Client byte rate is measured over multiple small windows (for e.g. 30 windows of 1 second each) in order to detect and correct quota violations quickly. Typically, having large measurement windows (for e.g. 10 windows of 30 seconds each) leads to large bursts of traffic followed by long delays which is not great in terms of user experience.
+</p>
+<h4>Quota overrides</h4>
+<p>
+    It is possible to override the default quota for client-ids that need a higher (or even lower) quota. The mechanism is similar to the per-topic log config overrides.
+    Client-id overrides are written to ZooKeeper under <i><b>/config/clients</b></i>. These overrides are read by all brokers and are effective immediately. This lets us change quotas without having to do a rolling restart of the entire cluster. See <a href="/ops.html#quotas">here</a> for details.
+
+</p>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/13803ae9/docs/ops.html
----------------------------------------------------------------------
diff --git a/docs/ops.html b/docs/ops.html
index 1604bf1..93640b0 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -297,6 +297,25 @@ Topic:foo	PartitionCount:1	ReplicationFactor:3	Configs:
 	Topic: foo	Partition: 0	Leader: 5	Replicas: 5,6,7	Isr: 5,6,7
 </pre>
 
+<h4><a id="quotas">Setting quotas</a></h4>
+It is possible to set default quotas that apply to all client-ids by setting these configs on the brokers. By default, each client-id receives an unlimited quota. The following sets the default quota per producer and consumer client-id to 10MB/sec.
+<pre>
+  quota.producer.default=10485760
+  quota.consumer.default=10485760
+</pre>
+
+It is also possible to set custom quotas for each client.
+<pre>
+> bin/kafka-configs.sh  --zookeeper localhost:2181 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048' --entity-name clientA --entity-type clients
+Updated config for clientId: "clientA".
+</pre>
+
+Here's how to describe the quota for a given client.
+<pre>
+> ./kafka-configs.sh  --zookeeper localhost:2181 --describe --entity-name clientA --entity-type clients
+Configs for clients:clientA are producer_byte_rate=1024,consumer_byte_rate=2048
+</pre>
+
 <h3><a id="datacenters">6.2 Datacenters</a></h3>
 
 Some deployments will need to manage a data pipeline that spans multiple datacenters. Our recommended approach to this is to deploy a local Kafka cluster in each datacenter with application instances in each datacenter interacting only with their local cluster and mirroring between clusters (see the documentation on the <a href="#basic_ops_mirror_maker">mirror maker tool</a> for how to do this).
@@ -601,6 +620,12 @@ We pay particular we do graphing and alerting on the following metrics:
       <td>kafka.server:type=KafkaRequestHandlerPool,name=RequestHandlerAvgIdlePercent</td>
       <td>between 0 and 1, ideally &gt 0.3</td>
     </tr>
+    <tr>
+      <td>Quota metrics per client-id</td>
+      <td>kafka.server:type={Produce|Fetch},client-id==([-.\w]+)</td>
+      <td>Two attributes. throttle-time indicates the amount of time in ms the client-id was throttled. Ideally = 0.
+          byte-rate indicates the data produce/consume rate of the client in bytes/sec.</td>
+    </tr>
 </tbody></table>
 
 <h4><a id="new_producer_monitoring">New producer monitoring</a></h4>
@@ -848,6 +873,16 @@ The following metrics are available on new producer instances.
       <td>The average per-second number of record sends that resulted in errors for a topic.</td>
       <td>kafka.producer:type=producer-topic-metrics,client-id=([-.\w]+),topic=([-.\w]+)</td>
     </tr>
+    <tr>
+      <td>produce-throttle-time-max</td>
+      <td>The maximum time in ms a request was throttled by a broker.</td>
+      <td>kafka.producer:type=producer-topic-metrics,client-id=([-.\w]+)</td>
+    </tr>
+    <tr>
+      <td>produce-throttle-time-avg</td>
+      <td>The average time in ms a request was throttled by a broker.</td>
+      <td>kafka.producer:type=producer-topic-metrics,client-id=([-.\w]+)</td>
+    </tr>
 </tbody></table>
 
 We recommend monitor GC time and other stats and various server stats such as CPU utilization, I/O service time, etc.