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

Compare with Current View Page History

« Previous Version 2 Next »

Status

Current state: "Under Discussion"

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]

JIRA: KAFKA-5092

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

Motivation

Kafka 0.10 added the Timestamp as part of messages, which is particularly important for KStreams application. Currently that timestamp can be set by the broker, or by the producer. This addresses concerns with timestamps set at the producer level.
Currently, if the producer application does not join a timestamp to the ProducerRecord, such as in public ProducerRecord(String topic, K key, V value) , then the timestamp is inferred at send time being System.currentTimeMillis(). If a producer wants to explicitly set a timestamp for the record, it has got to use public ProducerRecord(String topic, Integer partition, Long timestamp, K key, V value). 

This interface is particularly dangerous as it indicates to the user that they should provide an Integer partition. They would have to dive into the code to know that partition can be null, and then will be inferred using the key. I also believe offering partition as an argument goes against the current belief that "the same keys always goes to the same partition". 

This KIP addresses all these concerns, by adding a friendlier more explicit ProducerRecordBuilder interface, which also enables Kafka to easily evolve the message format without adding constructors to the ProducerRecord interface. 

Public Interfaces

The ProducerRecord constructors will all be deprecated, except the main long explicit one:
Deprecated:

public ProducerRecord(String topic, Integer partition, K key, V value)
public ProducerRecord(String topic, K key, V value)
public ProducerRecord(String topic, V value)

Not deprecated (but ideally protected in the future?) 

public ProducerRecord(String topic, Integer partition, Long timestamp, K key, V value)

New interface:

public ProducerRecordBuilder()
public ProducerRecordBuilder withKey(K key)
public ProducerRecordBuilder withValue(V value)
public ProducerRecordBuilder withTimestamp(Long timestamp)
public ProducerRecordBuilder withTopic(String topic)
public ProducerRecordBuilder withForcedPartition(Integer partition)
public ProducerRecord build()

Proposed Changes

A simple implementation would be:

 

package org.apache.kafka.clients.producer;

public class ProducerRecordBuilder<K, V> {

    private String topic;
    private Integer partition;
    private K key;
    private V value;
    private Long timestamp;

    public ProducerRecordBuilder(){};

    public ProducerRecordBuilder withKey(K key){
        this.key = key;
        return this;
    }

    public ProducerRecordBuilder withValue(V value){
        this.value = value;
        return this;
    }

    public ProducerRecordBuilder withTimestamp(Long timestamp){
        this.timestamp = timestamp;
        return this;
    }

    public ProducerRecordBuilder withTopic(String topic){
        this.topic = topic;
        return this;
    }

    public ProducerRecordBuilder withForcedPartition(Integer partition){
        this.partition = partition;
        return this;
    }

    public ProducerRecord build(){
        return new ProducerRecord<>(topic, partition, timestamp, key, value);
    }
}

 

Compatibility, Deprecation, and Migration Plan

  • Deprecation of ProducerRecord partial constructors
  • Migrating existing programs to the builder interface should be straightforward

Rejected Alternatives

I had tried to just add partial constructors here: https://github.com/apache/kafka/pull/2800/files

But as Ismael noted, public ProducerRecord(String topic, Long timestamp, K key, V value)

Adding this constructor is quite dangerous because it's almost the same as the one that takes partition (the only difference is that one is a Long and the other is an Integer)

  • No labels