Versions Compared

Key

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

...

  • client should be able to publish record with a negative timestamp (to the topics that support that),
  • broker should accept and serve that record,
  • streams should not drop a record with a negative timestamp.

NO_TIMESTAMP (−1) problem

The broker uses −1 as a default value for missing timestamp. Which might be a correct value set by the user: Wednesday, December 31, 1969 11:59:59 PM UTC.

Proposed behavior is to change that semantics and use Long.MIN_VALUE for messages without timestamp for new topics.Streams will ignore negative record timestamp if a topic was made without negative timestamp support.

We may need a new timestamp type along with CreateTime and LogAppendTime to prevent legacy brokers from writing −1 (with meaning no timestamp) to the new topics.

Changes in binary message format

Current binary format:

 

Code Block
collapsetrue
XXXX XXXX = 8 bits

 1. XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX - baseOffset
    XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX
 2. XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX - batchLength
 3. XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX - partitionLeaderEpoch
 4. XXXX XXXX                               - magic (current magic value is 2)
 5. XXXX XXXX XXXX XXXX                     - attributes

    Compression
    000 - no comporession
    001 - gzip
    010 - snappy
    011 - lz4

       Timestamp
       0 - create time
       1 - log append time

         X - isTransactional
          X - isControlBatch
           XX XXXX XXXX - unused

 6. XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX - lastOffsetDelta
 7. XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX - firstTimestamp
    XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX
 8. XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX - maxTimestamp
    XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX
 9. XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX - producerId
    XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX
10. XXXX XXXX XXXX XXXX XXXX XXXX XXXX XXXX - producerEpoch
11. ...                                     - baseSequence
12. ...                                     - records

Proposed change:

Code Block
collapsetrue
 5. XXXX XXXX XXXX XXXX - attributes

    Compression
    000 - no comporession
    001 - gzip
    010 - snappy
    011 - lz4

       Timestamp
       0 - create time
       1 - log append time

         X - isTransactional
          X - isControlBatch
           X - isTimestampExtended →
            X XXXX XXXX - unused

That isTimestampExtended bit should be 1 for all new records.

Broker should convert old NO_TIMESTAMP=−1L to new NO_TIMESTAMP_EXTENDED=Long.MIN_VALUE.

Compatibility, Deprecation, and Migration Plan

What impact (if any) will there be on existing users?

If users do not set a record timestamp (producers) or do not care about record timestamp when consuming topic then there will be no impact.

In case existing users do read record timestamp they should be aware of that −1 may mean actual timestamp (or no timestamp depending on the case)No impact on current users, they should update their infrastructure in that order: Broker, Consumers, Producers.

If we are changing behavior how will we phase out the older behavior? 

...

A migration tool is not required. Users may write their own consumer/producer to copy all data from old topic to topic that supports negative timestamp.
It's up to the users how to threat −1 timestamp from the old topic: convert to Long.MIN_VALUE or rely on broker wall clock.

When will we remove the existing behavior?

.

When will we remove the existing behavior?

No need.

Changes from producer perspective

 Old Broker BehaviourNew Broker Behaviour

Old producer sends NO_TIMESTAMP (−1L)

isTimestampExtended = 0

(tick) Records this as NO_TIMESTAMP (−1L)(tick) Records this as NO_TIMESTAMP (Long.MIN_VALUE)

New producer sends NO_TIMESTAMP (Long.MIN_VALUE)

isTimestampExtended = 1

(error) Error or Records this as NO_TIMESTAMP (−1L)(tick) Records this as NO_TIMESTAMP (Long.MIN_VALUE)

New producer sends −1L ms

isTimestampExtended = 1

(error) Error or Records this as NO_TIMESTAMP (−1L)(tick) Records this as −1L ms

So broker should be updated first, before producers.

Changes from consumer perspective

 Old ConsumerNew Consumer

Record has TS −1L

isTimestampExtended = 0

(tick) Interpret as NO_TIMESTAMP (−1L)(tick) Interpret as NO_TIMESTAMP (Long.MIN_VALUE)
Record has TS −1L

isTimestampExtended = 1

(error) Error or NO_TIMESTAMP(tick) Interpret as −1L ms
Record has TS Long.MIN_VALUE

isTimestampExtended = 1

(error) Error or NO_TIMESTAMP(tick) Interpret as NO_TIMESTAMP (Long.MIN_VALUE)

So only new consumers will read records correctlyNo need.

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.

...