Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Motivation

When a Kafka producer begins generating recordssends a record to a topic, it must retrieve have metadata about the topic it's producing to, specifically to determine information about in order to determine the partition to which the record will be written. No be delivered. Therefore, an unknown topic's metadata must be fetched whenever it is encountered. In the producer's current incarnation, no prior knowledge of the working set of topics is provided by the client to the producer, therefore so all topic metadata is fetched requested on-demand as it is encountered.

For the majority of use-cases, where a fixed and/or manageable number of topics are processed, fetching topic metadata is a cost that's incurred upon startup but subsequently mitigated by maintaining a metadata cache of the topics' metadata. However, in the case where a large or variable number of topics are written, clients may encounter degenerate metadata fetching behavior that can severely limit processingdegraded performance that severely limits processing, or in degenerate timeout cases, behavior which impedes progress altogether.

There are three primary factors in the producer that hinder client processing when working with a large number of topics:

  1. The number of metadata RPCs generated.
  2. The response size of the metadata RPCs.
  3. Throughput constriction while fetching metadata.

For (1), an RPC is generated every time an uncached topic's metadata must be fetched. During periods when a large number of uncached topics are processed (e.g. producer startup), this can result in a large number of RPCs in that are may be sent out to the controller in a short period of time. Generally, if there's n unknown topics, then n metadata RPCs will be sent regardless to their proximity in time.

For (2), requests for metadata will also ask to refresh metadata about all known , cached topics. As the working set number of topics becomes large, this can will inflate the response size and processing to be quite large and require non-trivial processing. This further exacerbates (1) in that every subsequent message metadata request will result in more metadata being transmittedan increasing amount of data transmitted back to the client for every RPC.

For (3), fetching of a topic's metadata is a blocking process operation in the clientproducer, which is all the more surprising because it blocks in a function that's advertised as asynchronous! This means that a pipeline of records to be submitted for various uncached topics may will serially block on the fetching of a particular while fetching an individual topic's metadata.

In concert, these three factors amplify the negative effects of each other, and should be resolved in order to alleviate any topic scalability issues.

Public Interfaces

No public interfaces will be modified.

...

The first step to addressing the above changes is to make the fetching of metadata asynchronous within the producer. This directly fixes (3), and opens the path for resolving (1) by enabling the metadata requests to be batched together. The producer Since the producer's interface is asynchronous and it inherently batches the sending of records to partitions, so subjecting the metadata fetching to a value <= subset of the batching delay doesn't change the interaction or expectation expectations of the client. This change alone should be good enough to bring performance back to acceptable, pending verification.

To address (2), the producer can maintain maintains a staleness duration threshold for every topic, and but it does not act upon this for metadata fetching. Further optimization could be done to only request metadata updates for the topics whose staleness thresholds have been exceeded. A soft threshold could also be added such that best-effort fetching could be performed on a subset of the topics, so that metadata updates are staggered over time and performed in smaller batches.

Compatibility, Deprecation, and Migration Plan

...