You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 23 Next »

Status

Current stateUnder Discussion

Discussion threadhttp://markmail.org/thread/hpp4ucysifucwmgd

JIRA

Motivation

Currently, the Kafka cluster does not have the ability to throttle/rate limit producers and consumers. It is possible for a consumer to consume extremely fast and thus monopolize broker resources as well as cause network saturation. It is also possible for a producer to push extremely large amounts to data thus causing memory pressure and large IO on broker instances. We need a mechanism to enforce quotas on a per-user basis.

In this KIP, we will discuss a proposal to implement quotas in Kafka. We are proposing an approach that can used for both producer and consumer side quotas.

Public Interfaces

  1. Metrics - The Quota Metrics which will be captured on a per-clientId basis, will be exposed to JMX. These are new metrics and do not use codahale. Instead, they use KM (Kafka Metrics) which is a new metrics library written for Kafka. More details in the Metrics section below.
  2. Client Response - Clients could optionally handle a flag in the response indicating whether they were throttled. This serves to inform users about their quota status and is not an error code.

Proposed Changes

The changes are mainly focussed around Quota Policy, Distribution, Metrics, Quota Actions and config management.

Quota Policy

This section basically explains what dimensions we throttle on. The proposal is to throttle based on client IDs. Any client using the system presents a client id or consumer group (producer or consumer). Each client will receive a default quota which can be overridden on a per-client basis dynamically. In addition, there will be a quota reserved for clients not presenting a client id (for e.g. simple consumers not setting the id). This will default to an empty client id ("") and all such clients will share the quota for that empty id (which should be the default quota).

A quota will defined in terms of read/write bytes per second. Any client that has just joined the cluster will receive a default quota per broker (for e.g. 10MBps read, 5MBps write). We do expect that there will be some high volume clients that require more than the default quota. For such clients, we need to provide a mechanism to override their quotas. In short, we are proposing fixed quotas for everyone but the top k outliers which can justify custom quotas. If users violate their quota, we will throttle fetch/produce requests for them.

Producer side quotas are defined in terms of bytes written per second per client id. Consumer quotas as defined in terms of bytes read per second per client id. For example: if a client deletes their consumer offsets and "bootstraps" quickly, this should cause a "read bytes per second" violation and will throttle (slow down) that consumer group. It should have no effect on any other clients.

Quota Distribution

We need a mechanism to distribute a clients quota across all the brokers in a cluster. Our proposal is to define the bandwidth on a per-broker basis. Each client can publish a maximum of X MBps (configurable) per broker before it gets throttled. This approach assumes that client requests are properly distributed among all the broker instances which may not always be the case. However, this is much better than having a fixed cluster wide bandwidth per client because that would require a mechanism to share current quota usage per-client among all the brokers. This can be very tricky to implement and is outside the scope of this proposal. 

Quota Actions


How do the Kafka Server react when it detects a quota violation? In our proposal, the broker does not return an error rather it attempts to slow down the client exceeding it's quota. The behavior is subtly different for producer and consumer.

In case of producer, the broker will append the request batch to the log but will not return a response immediately. Instead, the response is inserted into a DelayedOperationPurgatory for a certain period of time before returning to the client. It is important that the append be done before delaying the client to avoid holding large batches in the broker heap and running out of memory.

In case of consumer, the action is very similar to a delayed fetch request. The request needs to be delayed before reading from the log to avoid running out of memory.

Here's how the code might look inside the Broker (in KafkaApi's or helper class).

 

 

Metrics

Kafka server has decided to migrate it's metrics to Kafka Metrics (KM). For more details read this. Kafka metrics maintains a Sensor for each measured metric. This can be configured with a "Quota" which is a bound of the min and max value of a metric. If recording a value causes a bound to get exceeded, a QuotaViolationException is thrown.

We will add a bytes-in/bytes-out rate sensor per clientId configured with 'N' 1 second windows. This is already supported in MetricConfig. We can make 'N' configurable and use a good default (10). This lets us precisely track the rate per-client per second. Having small windows makes the quota system very responsive. A quota will also be configured when creating these metrics. For default clients, the quota used will simply be the default quota value. For clients with overridden quotas, we can use the custom value. 

In order to fully support quotas, we need to make some changes to the metrics package as defined below. 

Delay time in QuotaViolationException

Upon Quota violation, we need to identify how long we should delay requests for the client. It seems natural to have the QuotaViolationException return this information. The great thing about this is that we can catch any QuotaViolationException in the Kafka server and it will have the delay time built into it. This gives us the ability to enforce quotas over any metric we track.

Here's an example of computing the delay time. Assume a quota of 5MBps with a 10 second window. There is a client producing at exactly 5MBps. If it suddenly produces a 15MB batch in the next second, this causes a quota violation because it will have produced

60MB over the last 10 seconds (5*9 + 15 = 60MB).

Delay Time = (overall produced in window - quotabound)/Quota limit per second 
 
In the example above, we will delay for 2 seconds. 
Delay Time = (60 - 50)/5 = 2 second delay.

 

The code in KafkaApis or helper class can be:

 

try { quota.record(numBytes, time.currentTimeMillis()); } catch(QuotaViolationException ex) { delayedOperation = DelayedOperation(ex.getDelayTime(), ...) quotaPurgatory.tryCompleteElseWatch(operation); }

 

 


Configuration Management

How do we manage the quota overrides and the default topic configs? Manually configuring brokers with these is painful. In this case, the ability to dynamically change configs without bouncing brokers is very useful. There is already a proposal/patch for dynamic configuration management by Joe Stein which we plan to leverage for distributing these quota configs. In the future, we also need to think about restricting access to these configs (so that customers cannot modify their own quotas) but that is a separate discussion.

Broker Interface

Quota enforcement in the broker is done by the QuotaManager. check() is expected to be called before processing a request. If check() returns false, the broker is expected to take some quota action; otherwise the request is processed as usual.

/**
 * This is the basic interface we need to implement in the Kafka broker. This class will record all events processed by the broker. 
 */
public interface QuotaManager {
    /** 
     * The function check should be called before processing any request. If this returns false, then some Quota action has to be taken.
     */
    <T extends RequestOrResponse> boolean check(T request);

    /**
     * This function is called after any client request has been processed and lets the implementation track information about the response. 
     * This is useful for consumer side quotas where we may not know the response sizes and number of messages before processing the request entirely.
     *
     * T : T can be any type of request or response. Typically (FetchRequest or ProducerRequest)
     * C : C is basically the response object. Typically (FetchResponse or ProducerResponse).
     *
     */
	<T extends RequestOrResponse, C extends RequestOrResponse> void onResponse(T request, C response);

	void shutdown();
}

One alternative (proposed by Jun) is to simply use the "fetchSize" parameter on the FetchRequest (per partition) when calling check() to determine if the request is violating the quota or not. This removes the need to have the onResponse method. 

Example usage in KafkaApis:

// Example for consumers
def handleFetchRequest() {
  val fetchRequest = request.requestObj.asInstanceOf[FetchRequest]
  if(!quotaManager.check(fetchRequest))
    // take quota actions

  // Notify the quota manager after the request has been processed
  def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) {
      val fetchResponse = FetchResponse(fetchRequest.correlationId, responsePartitionData)
      quotaManager.onResponse(fetchRequest, fetchResponse);
  }
}

// Example for producers
def handleProducerRequest(request: RequestChannel.Request) {
  val produceRequest = request.requestObj.asInstanceOf[ProducerRequest]

  if(!quotaManager .check(fetchRequest)) {
    // take quota actions after emptying the request buffer
    produceRequest.emptyData();
  }

  // the callback for sending a produce response
  def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) {
    val response = ProducerResponse(produceRequest.correlationId, responseStatus)
    quotaManager.onResponse(produceRequest, response);
  }
}

Tooling/Monitoring Changes

Along with this feature, we are proposing to add the following tools/admin operations:

  • Dynamically disable/enable quota enforcement for the entire cluster. Such a feature is very useful while rolling out this feature to production environments.
  • Ability to disable quotas on a per-client basis. For example: we may not want to throttle mirror makers.
  • Dynamically change quotas for any client id.


We also plan to expose the fraction of quota used on a per-client basis via JMX (0-100%, where 100 means throttled).

Custom Implementations

The interface has been kept very generic to allow multiple implementations of the quota policy. The default policy that we have proposed should suffice for the majority of use cases. Since we pass in the actual request and response objects to the QuotaManager, these implementations should have enough information to build complex policies if required.

Compatibility, Deprecation, and Migration Plan

Rejected Alternatives

Topic Based QuotasWe initially considered doing topic based quotas. This was rejected in favor or client based quotas since it seemed more natural to throttle clients than topics. Multiple producers/consumers can publish/fetch from a single topic. This makes it hard to eliminate interference of badly behaved clients. Topic based quotas also are harder to reason about when clients are accessing multiple topics (e.g. wildcard consumer). If quota for 1 topic is violated, how do we handle requests for the other topics?

Quota Distribution 

A) Instead of throttling by client id, we can consider throttling by a combination of "clientId, topic". The default quota is also defined on this tuple i.e. 10Mbps for each "clientId, topic". The obvious downside here is that a wildcard consumer can allocate almost infinite quota for itself if it subscribes to all topics. Producers can also be allocated very large quotas based on the number of topics they publish to. In practice, this approach needs to be paired with a cap on the maximum amount of data a client can publish/consume. But this cluster-wide cap again requires global co-ordination which is we mentioned is out of scope.

B) If we instead model the quotas on a per-topic basis, provisioned quota can be split equally among all the partitions of a topic. For example: If a topic T has 8 partitions and total configured write throughput of 8MBps, each partition gets 1Mbps. If a broker hosts 3 leader partitions for T, then that topic is allowed 3MBps on that broker regardless of the partitions that traffic is directed to. In terms of quota distribution, this is the most elegant model.

 

Quota Actions

A) Immediately return an error: This is the simplest possible option. However, this requires clients to implement some sort of a back off mechanism since retrying immediately will likely make things worse.

 

 

B) Delay the request and return error: If a client is currently exceeding its quota, we can park the request in purgatory for 'n' milliseconds. After the request expires, the broker can return an error to the client. This is a nice feature because it effectively throttles the client up to the request timeout and makes it less critical for them to implement backoff logic. After the error is caught by the client, they can retry immediately. Note that in case of producers, we should be careful about retaining references to large Message bodies because we could easily exhaust broker memory if parking hundreds of requests. The downside of this approach is that it requires the clients to catch quota violation errors and retry their requests.

We have decided against (A) because it relies on clients to correctly implement back off behavior. A badly behaved client can simply retry in a loop and cause network saturation. In case of (B), clients have to handle an additional error code for quota and also build retry logic. In addition, there is a possibility of producer data loss if the request does not succeed after a certain number of retries.

There is more context of the email thread on this particular issue.

 

 



  • No labels