Table of Contents |
---|
Status
Current state: "Under DiscussionVoting in progress"
Discussion thread: on mail-archives.apache.org
...
- org.apache.kafka.common.record
- org.apache.kafka.clients.producer
- org.apache.kafka.streams.processor
Proposed Changes
Add a topic property that specifies that topic may have a valid Allow negative timestamps.
First, we need to remove all checks for negative timestamps across the code:
- 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: −1 value for timestamps stays a special value in Kafka as "no timestamp", which means we do not have a valid way to express Wednesday, December 31, 1969 111969 11:59:59.999 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.
If you do need to express Wednesday, December 31, 1969 11:59:59.999. PM UTC consider shifting it by one millisecond.
In general, there should be is no reason to fail on broker/client side working with negative timestampsWe 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.
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?
...
If we need special migration tools, describe them here.
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.
No need.
When will we remove the existing behavior?
...
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.
Message flag "hasTimestamp"
...
- add a special boolean flag to message record "
hasTimestamp
", - write a migration tool that adds this flag to message with the negative timestamp to legacy messages,
- make sure clients know about that field and check them
Ignore −1 problem:
...
- interpret this value as a real timestamp
- interpret −1 as "no timestamp" and other values as a real timestamp (can we borrow 1 millisecond for our needs?).
Timestamp delta
...
In case it's known that you would need no more than 200 years back before Unix epoch, the possible solution is:
...
It may seem the easiest thing to do, nevertheless changing −1 semantics is a cleaner solution.
Topic property "may have a valid negative timestamps"
Add a topic property that specifies that topic may have a valid negative timestamps.
First, we need to remove all checks for negative timestamps across the code:
- 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.
Changes from producer perspective
Old Broker Behaviour | New Broker Behaviour | |
---|---|---|
Old producer sends NO_TIMESTAMP (−1L)
| Records this as NO_TIMESTAMP (−1L) | Records this as NO_TIMESTAMP (Long.MIN_VALUE) |
New producer sends NO_TIMESTAMP (Long.MIN_VALUE)
| Error or Records this as NO_TIMESTAMP (−1L) | Records this as NO_TIMESTAMP (Long.MIN_VALUE) |
New producer sends −1L ms
| Error or Records this as NO_TIMESTAMP (−1L) | Records this as −1L ms |
So broker should be updated first, before producers.
Changes from consumer perspective
Old Consumer | New Consumer | |
---|---|---|
Record has TS −1L
| Interpret as NO_TIMESTAMP (−1L) | Interpret as NO_TIMESTAMP (Long.MIN_VALUE) |
Record has TS −1L
| Error or NO_TIMESTAMP | Interpret as −1L ms |
Record has TS Long.MIN_VALUE
| Error or NO_TIMESTAMP | Interpret as NO_TIMESTAMP (Long.MIN_VALUE) |
So only new consumers will read records correctly.
Changes in binary message format
Current binary format:
Code Block |
---|
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: let's use one of the reserved bits to indicate that timestamp can be negative.
Code Block |
---|
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
.