KIP-714 - Client Metrics and Observability - Apache Kafka - Apache Software Foundation
KIP-714 - Client Metrics and Observability - Apache Kafka - Apache Software Foundation
Status
Motivation
Proposed Changes
Overview
Client metrics subscription
Client identification and the client instance id
Mapping
Metrics naming and format
Encoding
Naming
Sparse metrics
Metric aggregation temporality
Serialized size
Compression
Client metrics and metric labels
Metric types
Standard producer metrics
Standard consumer metrics
Standard client resource labels
Broker-added resource labels
Client behavior
Threading
Connection selection
Client termination
Error handling
Java client dependencies
Receiving broker behavior
PushTelemetryRequest handling
Validation
Throttling and rate-limiting
Metrics subscription
Public Interfaces
Kafka Protocol Changes
Metrics serialization format
SubscriptionId
Client telemetry receiver
Admin API
Client API
Client configuration
Client metrics configuration
New error codes
Metrics
Security
Tools
kafka-configs.sh
kafka-client-metrics.sh
List all client metrics configuration resources
Describe a client metrics configuration resources
Create a client metrics configuration resource, generating a unique name
Create a client metrics configuration resource for all Python v1.2.* clients
Delete a client metrics configuration resource
Example workflows
Proactive monitoring
Reactive monitoring
Future work
Compatibility, Deprecation, and Migration Plan
What impact (if any) will there be on existing users?
Clients
Broker
If we are changing behavior how will we phase out the older behavior?
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 1/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
If we need special migration tools, describe them here.
When will we remove the existing behavior?
Rejected Alternatives
Create a new set of client telemetry metrics
Send metrics out-of-band directly to collector or to separate metric cluster
Produce metrics data directly to a topic
Dedicated Metrics coordinator based on client instance id
JMX
Combine with Dynamic client configuration
Use native Kafka protocol framing for metrics
Status
Current state: Approved
Discussion thread: here and now here
JIRA: KAFKA-15601
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Being able to centrally monitor and troubleshoot problems with Kafka clients is becoming increasingly important as the use of
Kafka is expanding within organizations as well as for hosted Kafka services. The typical Kafka client user is now an application
owner with little experience in operating Kafka clients, while the cluster operator has profound Kafka knowledge but little insight in
the client application. This is of particular importance for hosted Kafka services.
Troubleshooting Kafka problems is currently an organizationally complex issue, with different teams or even organizations running
the client applications and the brokers. While some organizations may already have custom collection of existing client metrics in
place, most do not and metrics are typically not available when the problem needs to be analysed. Enabling metrics after-the-fact
may not be possible without code change to the application, or at least a restart, which typically means the required metrics data is
lost.
While the broker already tracks request-level metrics for connected clients, there is a gap in the end-to-end monitoring when it
comes to visibility of client internals, be it queue sizes, internal latencies, error counts, and application behavior such as message
processing rates. These are covered by Kafka client metrics.
This proposal aims to provide a generic and standardized interface through which a cluster operator can request metrics, clients
can push metrics to the broker, a plugin interface for how those metrics are handled by the broker, and a minimum set of standard
pre-defined metrics that a Kafka client should expose through this interface. A supporting client should have metrics enabled by
default and not require any extra configuration, and will provide at least the required set of standard metrics as outlined later in this
proposal.
One of the key goals of this KIP is to have the proposed metrics and telemetry interface generally available and enabled by default
in all of the mainstream Kafka clients, not just the Java client in the Apache Kafka project, allowing troubleshooting and monitoring
as needed without interaction from cluster end-users. While metrics are to be enabled by default on the clients, the brokers still
need to be configured with a metrics plugin, and metrics subscriptions must be configured on the cluster before any metrics are
sent and collected. The default configuration for an Apache Kafka cluster does not include any metrics subscriptions.
This KIP defines two kinds of metrics: standard metrics and required metrics.
A standard metric is a Kafka client metric which clients that support this KIP should support. The standard metrics are a subset of
the Kafka client metrics, selected because they measure concepts which are likely to apply well to any client which implements the
Kafka protocol. The Apache Kafka Java client does by definition support all of the standard metrics. Other client implementations
should support standard metrics which conceptually make sense.
A required metric is a Kafka client metric which clients that support this KIP must support. The required metrics are a subset of the
standards metrics, selected because they have the highest value for monitoring and troubleshooting client performance and whose
concepts apply well to all clients which implement the Kafka protocol. The Apache Kafka Java client does by definition support all
of the required metrics.
The Apache Kafka client provides the reference implementation of these metrics. To implement these metrics in other clients, the
definition of the metrics is provided by the Apache Kafka documentation and ultimately the code.
Defining standard and required metrics makes the monitoring and troubleshooting of clients from various client types because the
operator can combine the same metric from all client types. For example, request latencies for all client types are reported in a
consistent way. Care has been taken to make the standard metrics as generic as possible, and they should fit most Kafka client
implementations.
User privacy is an important concern and extra care is taken in this proposal to not expose any information that may compromise
the privacy of the client user.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 2/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
Proposed Changes
Overview
This feature is made up of the following components:
GetTelemetrySubscriptions RPC - protocol request used by the client to acquire its initial Client instance ID and to
continually retrieve updated metrics subscriptions.
PushTelemetry RPC - protocol request used by the client to push metrics to any broker it is connected to.
Standard and required metrics - a set of standardized metrics that all supporting clients should provide.
AdminAPI config - the AdminAPI configuration interface with a new CLIENT_METRICS resource type is used to manage
metrics subscriptions.
Client metrics plugin / extending the MetricsReporter interface - a broker plugin interface that performs something
meaningful with the metrics. This plugin will typically forward the metrics to a time-series database. It is recommended that
any broker-side processing is kept to a minimum for scalability and performance reasons.
The metric collection is opt-in on the broker and opt-out on the client.
For metrics to be collected, a MetricsPlugin which implements the ClientTelemetry interface must be configured on the
brokers, and at least one metrics subscription must be configured through the Admin API. Only then will metrics subscriptions be
propagated to clients, and only then will clients push metrics to the broker. It is thus up to the cluster operator to explicitly enable
client metrics collection.
Because this KIP includes a new kind of ConfigResource, this KIP is only supported for KRaft clusters. This avoids doing the work
to store these new resources in ZooKeeper, given that ZooKeeper support is already deprecated.
Mapping
Mapping the client instance id to an actual application instance running on a (virtual) machine can be done by inspecting the
metrics resource labels, such as the client source address and source port, or security principal, all of which are added by the
receiving broker. This will allow the operator together with the user to identify the actual application instance.
Encoding
While future protocol-level requests could allow for other metrics formats, Kafka will only guarantee to support OpenTelemetry. It is
required for a client that claims support of this KIP to use the OpenTelemetry v0.19 protobuf format.
Metric payloads are encoded as OpenTelemetry MetricsData v1 protobuf objects.
Naming
This KIP introduces a new naming scheme for Kafka client metrics called telemetry metric names. Client metric subscriptions are
defined in terms of telemetry metric names. Telemetry metric names follow the OpenTelemetry conventions for metric naming.
Every existing Kafka client metric's name can be transformed into an equivalent telemetry metric name.
This proposal does not define any new client metrics, it just introduces a new scheme for naming them. The telemetry metric name
format is being introduced to give a hierarchical naming convention that meets OpenTelemetry rules, enabling interoperability with
that ecosystem.
The existing Kafka client metric names consist of two parts: a group name and a metric name. The telemetry metric name is
derived by concatenation of:
The prefix "org.apache.kafka."
The group name with "-metrics" removed, and every '-' replaced with '.'
The metric name with every '-' replaced with '.'
The following examples illustrate the derivation of the telemetry metric names from Kafka metric names:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 4/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
"connection-creation-rate", "org.apache.kafka.producer.connection.creation.rate"
group="producer-metrics"
"rebalance-latency-max", "org.apache.kafka.consumer.coordinator.rebalance.latency.max"
group="consumer-coordinator-metrics"
Other vendor or implementation-specific metrics can be added according to the following examples, using an implementation-
specific reverse domain name as the namespace to avoid clashes:
Metrics may also hold any number of attributes which provide the multi-dimensionality of metrics. These are similarly derived from
the tags of the Kafka metrics, and thus the properties of the equivalent JMX MBeans, replacing '-' with '_'. For example:
Sparse metrics
To keep metrics volume down, it is recommended that a client only sends metrics with a recorded value.
Serialized size
As an example, the serialized size prior to compression of all producer and standard metrics defined in this KIP for a producer
producing to 50 partitions gives approximately 100 kB.
Compression
As metric names and labels are highly repetitive, it is recommended that the serialized metrics are compressed prior to sending
them to the broker if the serialized uncompressed size exceeds 1000 bytes.
The broker will return a prioritized list of supported compression types in
the GetTelemetrySubscriptionsResponse.AcceptedCompressionTypes array, the client is free to pick any supported
compression type but should pick the first mutually supported type in the returned list. If the AcceptedCompressionTypes array is
empty the client must send metrics uncompressed. The default compression types list as returned from the broker should be: ZStd,
LZ4, GZip, Snappy.
The PushTelemetryRequest.CompressionType must then be set to the corresponding compression type value as defined for
MessageV2.
Preliminary tests indicate that a compression ratio up to 10x is possible for the standard metrics using ZStd.
Decompression of the metrics data will be performed by the broker prior to passing the data to the metrics plugin.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 5/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
Metric types
The OTLP metric data point types in the following tables correspond to the OpenTelemetry v1 metrics protobuf message types. A
short summary:
Sum - Monotonic total count meter (Counter). Suitable for total number of X counters, e.g., total number of bytes sent.
Gauge - Non-monotonic current value meter (UpDownCounter). Suitable for current value of Y, e.g., current queue count.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 7/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 8/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
If the received metrics have resource labels which clash with those added by the broker, the broker's overwrite the received values.
Client behavior
A client that supports this metric interface and identifies a supporting broker (through detecting at least
GetTelemetrySubscriptionsRequestV0 in the ApiVersionResponse) will start off by sending a GetTelemetrySubscriptionsRequest
with the ClientInstanceId field set to Null to one randomly selected connected broker to gather its client instance id, the subscribed
metrics, the push interval, accepted compression types, and so on. This handshake with a Null ClientInstanceId is only performed
once for a client instance's lifetime. Subsequent GetTelemetrySubscriptionsRequests must include the ClientInstanceId returned in
the first response, regardless of broker.
If a client attempts a subsequent handshake with a Null ClientInstanceId, the receiving broker may not already know the client's
existing ClientInstanceId. If the receiving broker knows the existing ClientInstanceId, it simply responds the existing value back to
the client. If it does not know the existing ClientInstanceId, it will create a new client instance ID and respond with that.
Upon receiving the GetTelemetrySubscriptionsResponse, the client shall update its internal metrics collection to match the received
subscription (absolute update) and update its push interval timer according to the received PushIntervalMs. The first metrics push
should be randomized between 0.5 * PushIntervalMs and 1.5 * PushIntervalMs. This is to ensure that not all clients start pushing
metrics at the same time after a cluster comes back up after some downtime.
If GetTelemetrySubscriptionsResponse.RequestedMetrics indicates that no metrics are desired (RequestedMetrics is Null), the
client should send a new GetTelemetrySubscriptionsRequest after the PushIntervalMs has expired. This is to avoid having to
restart clients if the cluster metrics configuration is disabled temporarily by operator error or maintenance such as rolling upgrades.
The default PushIntervalMs is 300000 ms (5 minutes).
If GetTelemetrySubscriptionsResponse.RequestedMetrics is non-empty but does not match any metrics the client provides, then
the client should send a PushTelemetryRequest at the indicated PushIntervalMs interval with an empty metrics blob. This is
needed so that a broker metrics plugin can differentiate between non-responsive or buggy clients and clients that don't have
metrics matching the subscription set.
Threading
The client will re-use the existing threads that are used for network communication. The existing logic in the poll method of the
NetworkClient class issues internal network requests for metadata when needed. The NetworkClient class has been
augmented to now also issue internal network requests for telemetry based on the poll interval in the subscription.
The threads that invoke the NetworkClient's poll method are:
KafkaAdminClient: the "admin client runnable" thread
KafkaConsumer: (existing code): both the "heartbeat" and the application threads
KafkaConsumer: (consumer threading refactor code as a part of KIP-848 effort): the "background" threads
KafkaProducer: the "sender" thread
Connection selection
The client may send telemetry requests to any broker, but shall prefer using an already available connection rather than creating a
new connection to keep the number of cluster connections down.
It should also keep using the same broker connection for telemetry requests until the connection goes down, at which time it may
choose to reconnect and continue using the same broker, or switch over to another broker connection. Using a persistent
connection for PushTelemetryRequests is important so that metrics throttling can be properly performed by the receiving broker,
and also avoids maintaining metrics state for the client instance id on multiple brokers.
Client termination
When a client with an active metrics subscription is being shut down, it should send its final metrics without waiting for the
PushIntervalMs time.
To avoid the receiving broker’s metrics rate-limiter discarding this out-of-profile push, the
PushTelemetryRequest.Terminating field must be set to true. A broker must only allow one such unthrottled metrics push
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 9/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
for each combination of client instance ID and SubscriptionId.
In the event that the client's metric subscription has changed and the final metrics push fails with error code
UNKNOWN_SUBSCRIPTION_ID , the terminating client can choose to obtain a new subscription ID by sending a
GetTelemetrySubscriptionsRequest and then immediately sending a PushTelemetryRequest with the Terminating flag set to true, or
it can choose to abandon sending a final metrics push.
Error handling
The following error codes can be returned in PushTelemetryResponse .
INVALID_REQUEST (42) Client sent a Log an error and stop pushing metrics. This is viewed as
PushTelemetryRequest a problem in the client implementation of metrics
when it has already serialization that is not likely to be resolved by retrying.
sent a Terminating
request.
INVALID_RECORD (87) Broker failed to decode Log an error and stop pushing metrics. This is viewed as
or validate the client’s a problem in the client implementation of metrics
encoded metrics. serialization that is not likely to be resolved by retrying.
TELEMETRY_TOO_LARGE (NEW) Client sent a Reduce the size of the metrics payload so its size does
PushTelemetryRequest not exceed
larger than the GetTelemetrySubscriptionsResponse.TelemetryMaxBytes.
maximum size the
broker will accept.
Retries should preferably be attempted on the same broker connection, in particular for UNKNOWN_SUBSCRIPTION_ID, but
another broker connection may be utilized at the discretion of the client.
How errors and warnings are propagated to the application is client- and language-specific. Simply logging the error is sufficient.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 10/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
If GetTelemetrySubscriptionsRequest.ClientInstanceId is Null the broker will generate a unique id for the client and
return it in the response. Subsequent requests from the client to any broker must have the ClientInstanceId set to this returned
value.
The broker should add additional metrics labels to help identify the client instance before forwarding the metrics to an external
system. These are labels such as the namespace, cluster id, client principal, client source address and port. As the metrics plugin
may need to add additional information on top of this, the generic metrics receiver in the broker will not add these labels but rely on
the plugins to do so. This avoids deserializing and serializing the received metrics multiple times in the broker.
See Broker-added labels below for the list of labels that should be added by the plugin.
The broker only reports support for GetTelemetrySubscriptions and PushTelemetry requests in its ApiVersionResponse if it has a
MetricsReporter that implements the ClientTelemetry interface. This means that clients will not attempt to push metrics to
brokers that are not capable of receiving them. If there is no client metrics receiver plugin configured on the broker, the client will
not send GetTelemetrySubscriptions or PushTelemetry RPCs, and existing behavior is preserved.
PushTelemetryRequest handling
Validation
Validation of the encoded metrics is the task of the ClientMetricsReceiver, if the compression type is unsupported the response will
be returned with ErrorCode set to UnsupportedCompressionType. Should decoding or validation of the binary metrics blob fail the
ErrorCode will be set to InvalidRecord.
Metrics subscription
Metrics subscriptions are configured through the standard Kafka Admin API configuration interface with the new config resource
type CLIENT_METRICS .
The ConfigResource has a name and is of type CLIENT_METRICS . The resource name does not have significance to the metrics
system other than to group metrics subscriptions in the configuration interface.
The configuration is made up of the following ConfigEntry names:
metrics - a comma-separated list of telemetry metric name prefixes, e.g.,
"org.apache.kafka.producer.node.request.latency.,
org.apache.kafka.consumer.coordinator.rebalance.latency.max". Whitespace is ignored.
interval.ms - metrics push interval in milliseconds. Defaults to 300000 ms (5 minutes) if not specified.
match - Client matching selector that is evaluated as a list of an anchored regular expressions (i.e., "something.*" is treated
as "^something.*$"). Any client that matches all of the selectors will be eligible for this metrics subscription. The regular
expressions are compiled and executed using Google RE2/J. Initially supported selectors are:
client_instance_id - CLIENT_INSTANCE_ID UUID string representation.
client_id - client's reported client.id in the GetTelemetrySubscriptionsRequest.
client_software_name - client software implementation name.
client_software_version - client software implementation version.
client_source_address - client connection's source address from the broker's point of view.
client_source_port - client connection's source port from the broker's point of view.
For example, using the standard kafka-configs.sh tool for create a metrics subscription:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 11/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
There is also a new kafka-client-metrics.sh tool which is described later that has easier syntax.
As the assumption is that the number of CLIENT_METRICS configuration entries will be relatively small (<100), all brokers with a
configured metrics plugin will monitor and cache the configuration entries for the CLIENT_METRICS resource type.
As a GetTelemetrySubscriptionsRequest is received for a previously unknown client instance id, the CLIENT_METRICS config
cache is scanned for any configured metric subscriptions whose match selectors match that of the client. The resulting matching
configuration entries are compiled into a list of subscribed metrics which is returned in
GetTelemetrySubscriptionsResponse.RequestedMetrics along with the minimum configured collection interval (this can be
improved in future versions by including a per-metric interval so that each subscribed metric is collected with its configured interval,
but in its current form longer-interval metrics are included “for free” if there are shorter-interval metrics in the subscription set). The
CRC32C checksum is also calculated based on the compiled metrics and is returned as the SubscriptionId in the response, as well
as stored in the per-client-instance cache on the broker to track configuration changes.
This client instance specific state is maintained in broker memory up to MAX(60*1000, PushIntervalMs * 3) milliseconds and is
used to enforce the push interval rate-limiting. There is no persistence of client instance metrics state across broker restarts or
between brokers.
Public Interfaces
Kafka Protocol Changes
These new RPCs are only supported on KRaft clusters.
GetTelemetrySubscriptionsRequestV0 {
ClientInstanceId uuid // UUID4 unique for this client instance.
// Must be set to Null on the first request, and
// from the first response for all subsequent req
}
GetTelemetrySubscriptionsResponseV0 {
ThrottleTimeMs int32 // The duration in milliseconds for which the req
// or zero if the request did not violate any quo
ErrorCode int16 // The error code, or 0 if there was no error.
ClientInstanceId uuid // Assigned client instance id if ClientInstanceI
SubscriptionId int32 // Unique identifier for the current subscription
AcceptedCompressionTypes Array[int8] // The compression types the broker accepts for P
// as listed in MessageHeaderV2.Attributes.Compre
// preference order from higher to lower. The Com
// present in the response from the broker, thoug
// client telemetry if none of the accepted compr
PushIntervalMs int32 // Configured push interval, which is the lowest
TelemetryMaxBytes int32 // The maximum bytes of binary data the broker ac
DeltaTemporality bool // If True; monotonic/counter metrics are to be e
// If False; monotonic/counter metrics are to be
RequestedMetrics Array[string] // Requested telemetry metrics prefix string matc
// Empty array: No metrics subscribed.
// Array[0] "*": All metrics subscribed.
// Array[..]: prefix string match.
}
PushTelemetryRequestV0 {
ClientInstanceId uuid // UUID4 unique for this client instance, as retr
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 12/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
SubscriptionId int32 // SubscriptionId from the GetTelemetrySubscripti
Terminating bool // Client is terminating.
CompressionType int8 // Compression codec used for .Metrics (ZSTD, LZ4
// Same values as that of the current MessageHead
Metrics binary // Metrics encoded in OpenTelemetry MetricsData v
}
PushTelemetryResponseV0 {
ThrottleTimeMs int32 // The duration in milliseconds for which the req
// or zero if the request did not violate any quo
ErrorCode int16 // The error code, or 0 if there was no error.
}
SubscriptionId
The SubscriptionId is a unique identifier for a client instance's subscription set, the id is generated by calculating a CRC32C of the
configured metrics subscriptions matching the given client including the PushIntervalMs, XORed with the ClientInstanceId. This
SubscriptionId is returned to the client in the GetTelemetrySubscriptionsResponse and the client passes that SubscriptionId in each
subsequent PushTelemetryRequest for those received metrics. If the configured subscriptions are updated and resulting in a
change for a client instance, the SubscriptionId is recalculated. Upon the next PushTelemetryRequest using the previous
SubscriptionId, the broker will find that the received and expected SubscriptionIds differ and it will return
UNKNOWN_SUBSCRIPTION_ID back to the client. When a client receives this error code it will immediately send a
GetTelemetrySubscriptionsRequest to retrieve the new subscription set along with a new SubscriptionId.
This mechanism provides a way for the broker to propagate an updated subscription set to the client and is similar to the use of
epochs in other parts of the protocol, but simplified in that no state needs to be persisted; the configured subscription set + client
instance id is the identifier itself.
/**
* A {@link MetricsReporter} may implement this interface to indicate support for collecti
*/
@InterfaceStability.Evolving
public interface ClientTelemetry {
/**
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 13/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
* Called by the broker to create a ClientTelemetryReceiver instance.
* This instance may be cached by the broker.
*
* This method will always be called after the initial call to
* {@link MetricsReporter#contextChange(MetricsContext)}
* on the MetricsReporter implementing this interface.
*
* @return
*/
ClientTelemetryReceiver clientReceiver();
}
@InterfaceStability.Evolving
public interface ClientTelemetryReceiver {
/**
* Called by the broker when a client reports telemetry. The associated request contex
* by the plugin to retrieve additional client information such as client ids or endpo
*
* This method should avoid blocking.
*
* @param context the client request context for the corresponding PushTelemetryReques
* @param payload the encoded telemetry payload as sent by the client
*/
void exportMetrics(AuthorizableRequestContext context, ClientTelemetryPayload payload)
}
@InterfaceStability.Evolving
public interface ClientTelemetryPayload {
/**
* Client's instance id.
*/
Uuid clientInstanceId();
/**
* Indicates whether the client is terminating and sending its last metrics push.
*/
boolean isTerminating();
/**
* Metrics data content-type / serialization format.
* Currently "application/x-protobuf;type=otlp+metrics0.19"
*/
String contentType();
/**
* Serialized uncompressed metrics data.
*/
ByteBuffer data();
}
Admin API
Add a new value CLIENT_METRICS to the enum org.apache.kafka.common.config.ConfigResource.Type . This
enables client metrics subscription configuration to be administered using the Kafka admin client as well as the kafka-
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 14/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
configs.sh tool.
Client API
Retrieve broker-generated client instance id, may be used by application to assist in mapping the client instance id to the
application instance through log messages or other means.
The client instance ids returned correspond to the client_instance_id labels added by the broker to the metrics pushed from the
clients. This should be sufficient information to enable correlation between the metrics available in the client, and the metrics
pushed to the broker.
The following method is added to the Producer, Consumer, and Admin client interfaces:
/**
* @return the client's assigned instance id used for metrics collection.
* @throws InterruptException If the thread is interrupted while blocked.
* @throws TimeoutException Indicates that a request timed out
* @throws KafkaException If an unexpected error occurs while trying to determin
* instance ID, though this error does not necessarily im
* consumer client is otherwise unusable.
* @throws IllegalArgumentException If the {@code timeout} is negative.
* @throws IllegalStateException If telemetry is not enabled ie, config `{@code enable.
* is set to `{@code false}`.
*/
public Uuid clientInstanceId(Duration timeout);
If the client has not yet requested a client instance id from the broker, this call may block up to the duration of the timeout. In the
event that the client instance id cannot be obtained within the timeout, the method throws
org.apache.kafka.common.errors.TimeoutException .
In addition, the following method is added to the KafkaStreams interface to give access to the client instance ids of the producers,
consumers and admin clients used by Kafka Streams:
/**
* @return The internal clients' assigned instance ids used for metrics collection.
*
* @throws IllegalArgumentException If {@code timeout} is negative.
* @throws IllegalStateException If {@code KafkaStreams} is not running.
* @throws TimeoutException Indicates that a request timed out.
* @throws StreamsException For any other error that might occur.
*/
public ClientInstanceIds clientInstanceIds(Duration timeout);
This method is only permitted when Kafka Streams is in state RUNNING or REBALANCING. In the event that Kafka Streams is not
in state RUNNING or REBALANCING, the method throws an IllegalStateException.
In the event that any of the client instance ids cannot be obtained within the timeout, the method throws
org.apache.kafka.common.errors.TimeoutException .
The new interface org.apache.kafka.streams.ClientInstanceIds is defined as follows:
/**
* Encapsulates the client instance ids used for metrics collection by
* producers, consumers and admin clients used by Kafka Streams.
*/
public interface ClientInstanceIds {
/**
* Get the client instance id of the admin client
*
* @return the client instance id
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 15/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
*
* @throws IllegalStateException If telemetry is disabled on the admin client.
*/
Uuid adminInstanceId();
/**
* Get the client instance ids of the consumers
*
* @return a map from thread key to client instance id
*/
Map<String, Uuid> consumerInstanceIds();
/**
* Get the client instance ids of the producers
*
* @return a map from thread key to client instance id
*/
Map<String, Uuid> producerInstanceIds();
}
Broker configuration
Configuration Description Values
telemetry.max.bytes The maximum size (after compression if compression is used) of int, default: 1048576, valid
telemetry pushed from a client to the broker. values: [1,...]
Client configuration
This applies to producers, consumers, admin client, and of course embedded uses of these clients in frameworks such as Kafka
Connect.
enable.metrics.push Whether to enable pushing of client metrics to the true (default) - The client will push
cluster, if the cluster has a client metrics subscription metrics if there are any matching
which matches this client. subscriptions.
false - The client will not push
metrics.
Following the usual convention, the string constant ENABLE_METRICS_PUSH_CONFIG ("enable.metrics.push") will be
added to CommonClientConfigs, ProducerConfig, ConsumerConfig, AdminClientConfig and StreamsConfig.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 16/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
metrics A list of telemetry metric name prefixes which An empty list means no metrics subscribed.
specify the metrics of interest.
A list containing just "*" means all metrics subscribed.
Otherwise, the list entries are prefix-matched against
the metric names.
interval.ms The client metrics push interval in milliseconds. Default: 300000 (5 minutes), minimum: 100, maximum:
3600000 (1 hour)
match The match criteria for selecting which clients the A list of key-value pairs.
subscription matches. If a client matches all of
these criteria, the client matches the The valid keys are:
subscription. client_instance_id - CLIENT_INSTANCE_ID
UUID string representation.
client_id - client's reported client.id in the
GetTelemetrySubscriptionsRequest.
client_software_name - client software
implementation name.
client_software_version - client software
implementation version.
client_source_address - client connection's
source address from the broker's point of view.
client_source_port - client connection's
source port from the broker's point of view.
The values are anchored regular expressions.
Metrics
The following new broker metrics should be added:
Security
Since client metric subscriptions are primarily aimed at the infrastructure operator that is managing the Kafka cluster it should be
sufficient to limit the config control operations to the CLUSTER resource.
There will be no permission checks on the PushTelemetryRequest itself.
Tools
kafka-configs.sh
The kafka-configs.sh tool can be used to administer client metrics configuration. The kafka-client-metrics.sh tool is
preferred because it is more useable.
A new entity-type of client-metrics is added.
For this entity type, the allowed configs are: interval.ms , metrics and match .
Some examples of use of kafka-configs.sh are shown in the next section for comparison with kafka-client-metrics.sh
.
kafka-client-metrics.sh
A new kafka-client-metrics.sh tool is added which provides a simpler interface for administering client metrics
configuration resources than using kafka-configs.sh directly.
Here's the command-line syntax summary.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 18/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
--generate-name Generate a UUID to use as the name.
--help Print usage information.
--interval The metrics push interval in milliseconds.
--match Matching selector ‘k1=v1,k2=v2’. The following
is a list of valid selector names:
client_instance_id
client_id
client_software_name
client_software_version
client_source_address
client_source_port
--metrics Telemetry metric name prefixes ‘m1,m2’.
--name <String> Name of client metrics configuration resource.
--version Display Kafka version.
Create a client metrics configuration resource for all Python v1.2.* clients
Example workflows
Example workflows showcasing how this feature may be used for proactive and reactive purposes.
Proactive monitoring
The Kafka cluster is configured to collect all standard metrics pushed by the client at an interval of 60 seconds, the metrics plugin
forwards the collected metrics to an imaginary system that monitors a set of well known metrics and triggers alarms when trends
go out of profile.
The monitoring system detects an anomaly for CLIENT_INSTANCE_ID=4e6fb54c-b8e6-4517-889b-e15b99c09c20 which
for more than 180 seconds has exceeded the threshold of 5000 milliseconds.
Before sending the alert to the incident management system, the monitoring system collects a set of labels that are associated with
this CLIENT_INSTANCE_ID, such as:
client.id
client_source_address and client_source_port on broker id X (1 or more such mappings based on how many connections
the client has used to push metrics).
principal
tenant
client_software_name and client_software_version
In case of consumer: group_id, group_instance_id (if configured) and the latest known group_member_id
In case of transactional producer: transactional_id
This collection of information, along with the triggered metric, is sent to the incident management system for further investigation or
aggregation, and provides enough information to identify which client and where the client is run. Further action might be to contact
the organization or team that matches the principal, transactional.id, source address and so on for further investigation.
Reactive monitoring
The Kafka cluster configuration for metrics collection (i.e., metrics subscriptions) is irrelevant to this use-case, given that a metrics
plugin is enabled on the brokers. The metrics plugin is configured to write metrics to a topic. A support system with an interactive
interface is reading from this metrics topic, and has an Admin client to configure the cluster with desired metrics subscriptions.
The application owner reports a lagging consumer that is not able to keep up with the incoming message rate and asks for the
Kafka operator to help troubleshoot. The application owner, who unfortunately does not know the client instance id of the
consumer, provides the client.id, userid, and source address.
The Kafka operator adds a metrics subscription for metrics matching prefix "org.apache.kafka.consumer." and with the
client_source_address and client_id as matching selector. Since this is a live troubleshooting case, the metrics push interval is set
to a low 10 seconds.
The metrics subscription propagates through configuration change notifications to all brokers which update their local metrics
subscription config cache and regenerates the SubscriptionId.
Upon the next PushTelemetryRequest or GetTelemetrySubscriptionsRequest from the clients matching the given source address
and client_id, the receiving broker sees that the SubscriptionId no longer matches its metrics subscription cache, the client
retrieves the new metrics subscription and schedules its next metrics push to a random value between PushIntervalMs * 0.5 ..
PushIntervalMs * 1.5.
Upon the next PushTelemetryRequest, which now includes metrics for the subscribed metrics, the metrics are written to the output
topic and the PushIntervalMs is adjusted to the configured interval of 10 seconds. This repeats until the metrics subscription
configuration is changed.
Multiple consumers from the same source address and using the same client_id may now be pushing metrics to the cluster. The
support system starts receiving the metrics and displays the matching client metrics to the operator. If the operator is able to further
narrow down the client instance through other information in the metrics it may alter the metrics subscription to match on that
client's client_instance_id. But in either case the metrics matching the given client.id are displayed to the operator.
The operator identifies an increasing trend in client.consumer.processing.time which indicates slow per-message processing in the
application and reports this back to the application owner, ruling out the client and Kafka cluster from the problem space.
The operator removes the metrics subscription which causes the next PushTelemetryResponse to return an error indicating that
the metrics subscription has changed, causing the client to get its new subscription which is now back to empty.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 20/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
Future work
Tracing and logging are outside the scope of this proposal but the use of OpenTelemetry allows for those additional two methods
within the same semantics and nomenclature outlined by this proposal.
When we add new client metrics to Kafka, we can choose whether to make them standard or required metrics in order to enhance
the ability to analyze and troubleshoot client performance.
We could also add proper support for histogram metrics in the client, and this would nicely fit with histograms in OpenTelemetry
metrics.
In network environments where there are network proxies (such as Kubernetes ingress) on the path between the client and broker,
it may be problematic obtaining the originating client's IP address. One way to address this in the future would be to support the
PROXY protocol in Kafka.
Configuration properties or extensions to the Metrics plugin interface on the broker to change the temporality is outside the scope
of this KIP and may be addressed at a later time as the need arises. For example, if a metrics back-end has a preference for a
particular temporality, it may be helpful to let it indicate that using the Metrics plugin interface so that the broker can use this
temporality when requesting metrics from the clients.
Broker
The Kafka admin needs to explicitly configure a Metrics reporter plugin that supports the new push-based telemetry interface. If no
such plugin is configured, the telemetry collection is disabled and there will be no user-facing changes. The default configuration
for the Kafka broker does not configure a plugin.
Depending on the metrics subscription patterns and intervals, there is likely to be increased CPU and network load as brokers will
receive, decompress and forward pushed client telemetry to the metrics plugin.
If we are changing behavior how will we phase out the older behavior?
No behavioral changes.
Rejected Alternatives
Create a new set of client telemetry metrics
An earlier version of this KIP introduced a new set of client telemetry metrics, completely independent of the existing Kafka client
metrics. This was rejected because it was felt that the lack of correspondence between the metrics available on the client and
those sent to the broker to support this KIP was confusing. For example, there were two independent ways of measuring latency
which did not match.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 21/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
JMX
Scales poorly, Java-specific, commonly mistaken for a trick bike.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 22/23
11/03/2024, 10:47 KIP-714: Client metrics and observability - Apache Kafka - Apache Software Foundation
No labels
4 Comments
Travis Bischel
Can a discussion thread be opened for this KIP? I think the goals of this KIP are great, but I think that this is better
served by clients making it simple for users to gather the metrics they want / are interested in – something that a KIP is
not necessary for. Also, metrics are more of an organizational concern, not a client / broker concern, and this KIP seems
to be addressing org. concerns by piping everything through Kafka. This makes sense if you're a Kafka operator, but
doesn't necessarily make sense if you're an observability team that is not running the Kafka stack.
I have a fuller writeup, so if this can be mailing list discussion, that'd be great, otherwise let me know and I'll post my
thoughts here. Thanks!
Ismael Juma
Travis, a discuss thread was started 11 days ago in the mailing list.
Travis Bischel
Thank you! I must have checked one or two days before that.
Philip Nee
Think I need a bit of clarification for the kafka-client-metrics.sh cli util here:
1. For list metrics,
a. do we want user to do: --list --name config-name-1 config-name2?
2. Do we actually need the --name tag for add/list/delete? I mean there's no other identifiers to be used, to perform
these actions?
3. We should provide an example usage here for clarity.
@ Kirk True @ Magnus Edenhill any suggestions here?
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability 23/23