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

Compare with Current View Page History

« Previous Version 43 Next »

 

Status

Current state:  Under Discussion 

Discussion threadhere 

JIRAKAFKA-4208

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

This KIP tries to address the following issues in Kafka.

In most message systems (JMS, QPID etc), streaming systems and most transport systems(HTTP, TCP), it is typical to have a concept of headers and payload.

The payload is traditionally for the business object, and headers are traditionally used for transport routing, filtering etc. Headers are most typically key=value pairs.

In its current state Kafka does not support the ability to have headers natively in its message/record format.

Examples where having separate supported custom headers becomes useful (this is not an exhaustive list).

  • Automated routing of messages based on header information between clusters
  • Enterprise APM tools (e.g. Appdynamics, Dynatrace) need to stitch in 'magic' transaction ids for them to provide end to end transaction flow monitoring.
  • Audit metadata to be recorded with the message, e.g. clientId that produced the record, unique message id, originating clusterId the message was first produced into for multi cluster routing. 
  • Business payload needs to be end to end encrypted and signed without tamper, but eco-system components need access to metadata to achieve tasks.  

Kafka currently has Record<K, V> structure which originally could be used to follow this semantic where by K could contain the headers information, and the V could be the payload.

  • Since message compaction feature it is no longer possible to add metadata to K, else compaction would treat each message as a different keyed message .
  • It is not currently possible to use value part and use some form of a wrapper e.g. Message<H, V>, as for compaction to perform a delete a record is sent with a NULL value, as such for where a delete record is sent using a message wrapper to carry the metadata would not work, as the value technically would no longer be null.

This issue has been flagged by many people over the past period in forums.

Further details and a more detailed case for headers can be seen here : A Case for Kafka Headers

Public Interfaces

This KIP has the following public interface changes:

  1. Add a new headers length and value (byte[]) to the core message format.
  2. Add a headers (Map<int, byte[]>) field to ProducerRecord and ConsumerRecord. A producer/interceptors will be able to set headers on a ProducerRecord. A consumer/interceptors will see the message headers (immutable) when it sees the messages.

  3. Add accessor methods on the Producer/ConsumerRecord void setHeader(int, byte[]) and a byte[] getHeader(int)
  4. Add ProduceRequest/ProduceResponse V3 which uses the new message format.
  5. Add FetchRequest/FetchResponse V3 which uses the new message format.
  6. The serialisation of the [int, bye[]] header set will on the wire using a strict format
  7. Each headers value will be custom serialisable by the interceptors/plugins that use the header.
  8. As int key based headers for compactness ranges will be reserved for different usages:
    1. client headers adopted by the kafka open source (not necessary the kafka open source owns it)
    2. broker headers adopted by the kafka open source (not necessary the kafka open source owns it)
    3. commercial vendor space
    4. custom inhouse

For more detail information of the above changes, please refer to the Proposed Changes section.

Proposed Changes

There are four options proposed before this proposal. This details our proposed solution of Option 1 described here. The other options Option 2, Option 3 and Option 4 are in the Rejected Alternatives section.

 

The advantages of this proposal are:
  • Adds the ability for producers to set standard header key=value value pairs
  • No incompatible client api change (only new methods)
  • Allows users to specify the serialisation of the header value per header
  • Compact key space
  • Provides a standardised interface to eco systems of tools that then can grow around the feature

The disadvantage of this proposal is:

  • Change to the message object

Add a headers field Map<int, byte[]> to both ProducerRecord and ConsumerRecord

  • Accessor methods of void setHeader(int, byte[]) and byte[] getHeader(int) added to interface of the ProducerRecord.
  • Accessor methods of byte[] getHeader(int) added to interface of the ConsumerRecord. (immutable as such no set required)

Wire protocol change - use attribute bit4 as flag for if headers present. and add (optional) header size and field to the message format

The below is for the core Message wire protocol change needed to fit headers into the message.

A key to this is to ensure the headers cause no overhead if not present.

  • The attributes flag bit is used to keep the message size the same as before if no headers are used

MessageAndOffset => Offset MessageSize Message
  Offset => int64  
  MessageSize => int32
  
  Message => Crc MagicByte Attributes Timestamp KeyLength Key HeadersLength Headers ValueLength Value
    Crc => int32
    MagicByte => int8  <---------------------- Bump up magic byte to 2
    Attributes => int8 <---------------------- Use Bit 4 as boolean flag for if headers present
    Timestamp => int64
    KeyLength => int32
    Key => bytes
    (optional) HeadersLength => int32 <--------------- NEW [optional] size of the byte[] of the serialised headers if headers
    (optional) Headers => bytes <--------------------- NEW [optional] serialised form of the headers Map<int, byte[]>
    ValueLength => int32
    Value => bytes


Wire protocol of the headers bytes (if above mentioned attributes bit flag is true)

The below is for the headers wire protocol.

  • The headers to be ordered by the int key (ascending)
    • Benefits are more prevalent on the server side
      • Allows faster access to lower numbered keys which can be reserved for server side set/accessed headers
      • Ability to skip/not read higher numbered keys used by clients.
      • Examples of such server side cases that would benefit:
Headers (bytes) => Set(Key, ValueLength, Value)
  Set =>
	Key => int32 <---------------------- NEW int key of the header
    ValueLength => int32 <-------------- NEW size of the byte[] of the serialised header value
    Value => bytes <-------------------- NEW serialised form of the header value
 

 

Example Key Allocation

As mentioned above ranges of keys would will be reserved for different usages, below is an example way this could be managed.

Kafka open source  (0x000)
  • 0x00000000-0x0000FFFF Kafka open source
  • 0x00010000-0x000FFFFF RESERVED
Local (0x001)
  • 0x00100000-0x0010FFFF Local use - Infrastructure
  • 0x00110000-0x0011FFFF Local use - Client
  • 0x00120000-0x0012FFFF Local use - Stack/Middleware
  • 0x00130000-0x0013FFFF Local use - Application
  • 0x00140000-0x0014FFFF Local use - User 
  • 0x00150000-0x001FFFFF RESERVED
Open (0x002)
  • 0x00200000-0x0020FFFF Open use - Infrastructure
  • 0x00210000-0x0021FFFF Open use - Client
  • 0x00220000-0x0022FFFF Open use - Stack/Middleware
  • 0x00230000-0x0023FFFF Open use - Application
  • 0x00240000-0x0024FFFF Open use - User 
  • 0x00250000-0x002FFFFF RESERVED
Testing (0x003)
  • 0x00300000-0x0030FFFF Testing - Infrastructure
  • 0x00310000-0x0031FFFF Testing - Client
  • 0x00320000-0x0032FFFF Testing - Stack/Middleware
  • 0x00330000-0x0033FFFF Testing - Application
  • 0x00340000-0x0034FFFF Testing - User 
  • 0x00350000-0x003FFFFF RESERVED
Reserved
  • 0x00400000-0xFFFFFFFF RESERVED

 

From this perspective we could have the following:
  • A kafka header used in the kafka open source distribution could have the value of 0x00000010 (10).
  • A header used internally at organisation X would fall under Local. If it was something used by infrastructure, like tracing, that could use 0x00100010 (1048592)
  • A header used by an open source project somebody is working on, that they just want to put out there (without coordination) could start using a value of 0x00220010 (2228240) if it was a plugin for annotating the location on a message.
  • An application that was testing whether it can use headers for a new feature it's developing could pick a header with key 0x0034010 (3407888).
  • The Kafka open source number space is coordinated via the Apache Kafka opensource project.  A class would list all possible headers, their numbers and their string equivalents (for output/logging/etc).
  • A Local organisation is in charge of coordinating it's local number space. It would be in charge of writing a text file, a class or a service to coordinate who gets what numbers when.
  • In the open internet you can grab any number in the Open space but should expect no guarantees that other people may or may not be using that number.
  • When you're doing testing you can safely grab a number in the testing space and be assured that you won't collide with an official system header.  It's still possible to collide with other testing headers, but no production system should depend on these headers.
Sample register that would end up having in the open source space.
keynamedescriptionbyurl
1client.idproducers client idApache Kafkasome url to a document about it
2cluster.idcluster id of where the message first originatedApache Kafkasome url to a document about it
3correlation.idcorrelation id for where a message is for mutex response from a requestApache Kafkasome url to a document about it
     
2100001new.relicstores the transaction linking guid for transaction sticking by new relicNew Relicsome url to a document about it
2100002appdynamicsstores the transaction linking guid for transaction stiching by app dynamicsAppDynamicssome url to a document about it

 

To assit and help ensure ease of use and uniformity a constants class should be kept and updated with the above (similar to how java sql codes work) e.g.

package org.apache.kafka.common.config;

public class KafkaHeaderKeys
{

   public static final int CLIENT_ID_KEY = 1;
   public static final int CLUSTER_ID_KEY = 2;
   public static final int CORRELATION_ID_KEY = 3;
   
}

 

Sample register that would end up having local (In-house) custom per organisation
keynamedescriptionbynotes
1114000app.nameig's unique app name for the producerIGsome internal document about it
1050000charge.tagtag to make per message chargebacks forIGsome internal document about it

Compatibility, Deprecation, and Migration Plan

  • Current client users should not be affected, this is new api methods being added to client apis
  • Message version allows clients expecting older message version, would not be impacted 
    • older producers simply would produce a message without headers
      • new consumer would simply get a message with empty headers
    • older consumers simply would consume a message oblivious to there being any headers
  • Message version migration would be handled as like in KIP-32

Out of Scope

Some additional features/benefits were noticed and discussed on the above but are deemed out of scope and should be tackled by further KIPS.

  • Core message size reduction
    • remove overhead of 4 bytes for KeyLength when no headers using attributes bit
    • reduce overhead of 4 bytes for KeyLength by using variable length encoded int
    • reduce overhead of 4 bytes for ValueLength by using variable length encoded int
  • Broker side interceptors
    • with headers we could start introducing broker side message interceptors to append meta data or handle messages
  • Single Record consumer API
    • There is many uses cases where single record consumer/listener api is more user friendly - this is evident by the fact spring kafka have already created a wrapper, it would be good to support this natively.

Rejected Alternatives

Map<String, String> Headers added to the ProducerRecord

The concept is similar to the above proposed but with a few more disadvantages.

  • Benefits
    • Adds the ability for producers to set standard header key=value string value pairs
    • No incompatible client api change (only new methods)
    • Allows users to specify the serialisation of the key=value map (String(&=), JSON, AVRO).
    • Provides a standardised interface to eco systems of tools can grow around the feature
  • Disadvantages
    • Change to the message object
    • String key cause a large key, this can cause a message payload thats of 60bytes to be dwarfed by its headers
    • String value again doesn't allow for compact values, and restricts that a value must be a String
    • Not able to use headers on the broker side with custom serialisation
Value Message Wrapper - Message<H, P>

This concept is the current defacto way many users are having to temporally deal with the situation, but has some core key issues that it does not resolve.

  • Benefits
    • This will cause no broker side changes, and or message format changes
  • Disadvantages
    • This would not work with compaction where headers are needed to be sent on delete record which then would not deliver on many of the requirements.
    • Couple Serialisation and Deserialisation of the value for both the header and payload.
ProducerRecord<K, H, V>, ConsumerRecord<K, H, V>

The proposed change is that headers are Map<String, String> only, this alternative is that headers can be of any type denoted by H

  • Benefits
    • Complete customisation of what a header is.
  • Disadvatages
    • As generics don't allow for default type, this would cause breaking client interface compatibility if done on Producer/ConsumerRecord.
      • Possible work-around would be to have HeadersProducer/ConsumerRecord<K, H, V> that then Producer/ConsumerRecord extend where H is object, this though becomes ugly fast if kept for a time or would require a deprecation / refactor v2 period.

  • No labels