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

Compare with Current View Page History

« Previous Version 31 Next »

Status

Current state: Under Discussion

Discussion thread: here

JIRA: KAFKA-2511

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

Motivation

This KIP is try to address the following issue in Kafka.

  1. Log retention might not be honored: Log retention is currently at the log segment level, and is driven off the last modification time of a log segment. This approach does not quite work when a replica reassignment happens because the newly created log segment will effectively have its modification time reset to now.
  2. Log rolling might break for a newly created replica as well because of the same reason as (1).

Besides that, the KIP will also facilitate some use cases such as streaming processing where a timestamp is needed.

This KIP is a distilled/improved version of an earlier discussion that we started.

This KIP is preferably to be implemented with KIP-31 if possible to avoid changing wire protocol twice.

Public Interfaces

There are a few options to achieve the goals. Each has their own pros and cons. Please see the details below.

Proposed Changes

Option 1 - Add both CreateTime and LogAppendTime to message format

Wire protocol change

MessageAndOffset => MessageSize Offset Message
  MessageSize => int32
  Offset => int64
  
  Message => Crc MagicByte Attributes Timestamp KeyLength Key ValueLength Value
    Crc => int32
    MagicByte => int8
    Attributes => int8
    CreateTime => int64    <---------------------- NEW
    LogAppendTime => int64 <---------------------- NEW
    KeyLength => int32
    Key => bytes
    ValueLength => int32
    Value => bytes

Add CreateTime and LogAppendTime field to message

  • CreateTime

    • CreateTime will be set by the producer and will not be changed afterward.
    • CreateTime accuracy is millisecond.
    • For compressed message, the CreateTime of the wrapper message will be the CreateTime of the first compressed message.
  • LogAppendTime
    • The LogAppendTime will be assigned by broker upon receiving the message. If the message is coming from mirror maker, the original CreateTime will be maintained but the LogAppendTime will be changed by the target broker.
    • The LogAppendTime will be used to build the log index.
    • The LogAppendTime accuracy is millisecond
    • The LogAppendTime of the outer message of compressed messages will be the latest LogAppendTime of all its inner messages.
      • If the compressed message is not compacted, the relative offsets of inner messages will be contiguous and share the same LogAppendTime.
      • If the compressed message is compacted, the relative offsets of inner messages may not be contiguous. Its LogAppendTime will be the LogAppendTime of the last inner message.
    • The followers will not reassign LogAppendTime but simply update an in memory lastAppendedLogAppendTime and append the message to the log.
    • To handle leader migration where new leader has slower clock than old leader, all the leader should append max(lastAppendedTimestamp, currentTimeMillis) as the timestamp

A corner case for LogAppendTime(LAT) - Leader migration:

Suppose we have broker0 and broker1. Broker0 is the current leader of a partition and broker1 is a follower. Consider the following scenario:

  1. message m0 is produced to broker 0 at time T0 (LAT = T0, T0 is the clock on broker 0).
  2. broker1 as a follower replicated m0 and appended to its own log without changing the LogAppendTime of m0.
  3. broker0 went down and broker 1 become the new leader.
  4. message m1 is produced to broker 1 at time T1 (LAT = T1, T1 is the clock on broker 1).

In step 4, it is possible that T1 < T0 because of the time difference on two brokers. If we naively take T1 as the timestamp of m1, then the timestamp will be out of order. i.e. a message with earlier timestamp will show later in the log. To avoid this problem, at step 4, broker 1 can take max(T1, T0) as the timestamp for m1. So the timestamp in the log will not go backward. Broker 1 will be using T0 as timestamp until its own clock advances after T0.

To be more general, when a message is appended, broker (whether leader or follower) should remember the timestamp of the last appended message, if a later appended message has a timestamp earlier than the timestamp of last appended message, the timestamp of last appended message will be used.

Change time based log rolling and retention to use LogAppendTime

Time based log rolling and retention currently use the file creation time and last modified time. This does not work for newly created replicas because the time attributes of files is different from the true message append time. The following changes will address this issue.

  • The time based log rolling will be based on the timestamp of the first message in the log segment file.
  • The time based log retention will be based on the timestamp of the last message in the log segment file.

ConsumerRecord / ProducerRecord format change

  • Add a CreateTime field to ProducerRecord. This field can be used by application to set the send time. It will also allow mirror maker to maintain the send time easily.
  • Add both CreateTime and LogAppendTime to ConsumerRecord.
    • The CreateTime is useful in use cases such as stream processing
    • The LogAppendTime is useful in use cases such as log rolling and log retention.

Option 2 - Add only LogAppendTime to the message

See Rejected Option Section

Option 3 - Add only CreateTime to the message

While the time based log index has to be based on LogAppendTime, there is some concern about exposing the LogAppendTime (which is a broker internal concept) to user. And there will also be some per message overhead for two timestamps.

So this approach includes only CreateTime in the message format.

Wire protocol change

Change the MessageAndOffset format to:

MessageAndOffset => MessageSize Offset Message
  MessageSize => int32
  Offset => int64
  
  Message => Crc MagicByte Attributes Timestamp KeyLength Key ValueLength Value
    Crc => int32
    MagicByte => int8
    Attributes => int8
    CreateTime => int64 <---------------------- NEW
    KeyLength => int32
    Key => bytes
    ValueLength => int32
    Value => bytes

Build time based log index using LogAppendTime

The broker will still build time based index using LogAppendTime, LogAppendTime will be only in the time index file, but not in message format. i.e. not exposed to user.

When the broker will append a time index file entry for a message when:

  1. The message is the first message of a log segment.
  2. The message is the last message of a log segment.
  3. The message is the first message received in the minute.

Let replicas to also fetch log index file

Because LogAppendTime is not included in the message format. With current replication design, followers will not be able to get the LogAppendTime from leader. In order to make log retention and log rolling policy work, the LogAppendTime needs to be propagated from leader to followers.

In this option, the LogAppendTime only exist in the time index file, therefore when followers fetch data from the leader, they have to replicate the time index file as well.

There are a few requirements here:

  1. Unlike log index file, the time index file should not be rebuilt from local log when it crashes, but should always be fetched from the current leader, just the same as actual data. Otherwise we may have different time index on different replicas.
  2. To ensure the log segments are identical on both leader and followers, we should always have a time index entry for the first message in a log segment.
  3. In order to make the time based log retention work, we need the timestamp entry for the last message in a log segment.
  4. When we truncate the messages in log segment files, we need to truncate entries in the time index files as well.

To replicate the log index entry as well, we can add the log index entry to FetchResponse, so the fetch response will become

FetchResponse format for replication
FetchResponse => [TopicName [Partition ErrorCode HighwaterMarkOffset MessageSetSize MessageSet [TimeIndexEntry]]]
  TopicName => string
  Partition => int32
  ErrorCode => int16
  HighwaterMarkOffset => int64
  MessageSetSize => int32
  TimeIndexEntry => LogAppendTime Offset <------------- new, the time index entry in the message set, one partition might contain multiple entries. The array will always be empty if the fetch request is not from followers.
    LogAppendTime => int64
    Offset => int32

Option discussions with use cases

This section discusses how the three options work with a few use cases. Option 2 is in the rejected option section.

Options comparison

Use cases

Option 1

(Message contains CreateTime + LogAppendTime)

option 2

(Message contains LogAppendTime only)

option 3

(message contains CreateTime only, brokers keep LogAppendTime in log index)

Comparison
Mirror MakerBroker overrides the LAT and keep the CT as is.Broker overrides the LATBroker keep the CT as. And add index entry with LAT to the log index file.

Option 1 provides the most information to user. The only concern is whether we should expose LAT to user.

Option 2 loses the CreateTime information.

Option 3 have same amount information as option 1 from broker point of view. From user point of view, it does not expose the LAT.

Log RetentionBroker will use the LAT of the last message in a segment to enforce the policy.Same as option 1.

Broker will use the LAT of the last entry in the log index file to enforce the retention policy. Because the leader is the source of truth for LAT, followers need to get the LAT from leader when they replicate the messages. That means we need to introduce a new wire protocol to fetch the time based log index file as well.

When log recovery happens, the rebuilt time index would have different LAT from the actual arrival time of the messages in the log. And the LAT in the index file will be very close, or even the same.

Option 1 and option 2 can work with existing replication design and solve the log retention issue we have now.

Option 3 alone can not solve the problem we have now. We need additional replication protocol to solve the log retention problem.

Log rollingBroker will use the LAT of the first message in a log segment to enforce the policy.Same as option 1.

Broker will use the LAT of the first entry in the log index file to enforce the retention policy. Similar to the log retention case, the followers needs to replicate the time index as well.

The log recovery happens, the log rolling might not be honored either.

Option 1 and option 2 solves the log rolling issue.

Option 3 does not solve the problem and needs additional replication protocol.

Stream processingApplications don't need to include the CreateTime in the payload but simply use the CreateTime field.Applications have to put CreateTime into the payload.Applications don't need to include the CreateTime in the payload but simply use the CreateTime field.The benefit of having a CreateTime with each message rather than put it into payload is that application protocol can be simplified. It is convenient for the infrastructure to provide the timestamp so there is no need for each application to worry about the timestamp.
Latency measurementUser can get End2End latency and lag in time.User can get the lag in time.User can get End2End latency.Option 1 has most information for user.
Search message by timestamp.Detail discussion in KIP-33Detail discussion in KIP-33Detail discussion in KIP-33Detail discussion in KIP-33

Mirror maker case in detail

The behavior of broker for all the options are the same: The broker will always override the LogAppendTime(if exists) when message arrives the broker and keep the CreateTime(if exists) untouched. 

The broker does not distinguish mirror maker from other producers. The following example explains what will the timestamp look like when there is mirror maker in the picture.(CT - CreateTime, LAT - LogAppendTime)

  1. Application producer produces message at T0. ( [CT = T0, LAT = -1] ) 
  2. The broker in cluster1 received message at T1 = T0 + latency1 and appended the message to the log. Latency1 includes linger.ms and some other latency. ( [CT = T0, LAT = T1] )
  3. Mirror maker copies the message to broker in cluster2. ( [CT = T0, LAT = T1] )
  4. The broker in cluster2 receives message at T2 = T1 + latency2 and appended the message to the log. ( [CT = T0, LAT = T2] )

The CreateTime of a message in source cluster and target cluster will be same. i.e. the timestamp is passed across clusters. 

The LogAppendTime of a message in source cluster and target cluster will be different.

Discussion: should we use CreateTime OR LogAppendTime for log retention and time based log rolling?

To discuss the usage of CreateTime and LogAppendTime, it is useful to summarize the latency pattern of the messages flowing through the pipeline. The latency can be summarized to the following pattern:

  1. the messages flow through the pipeline with small latency.
  2. the messages flow through the pipeline with similar large latency.
  3. the messages flow through the pipeline with large latency difference.

Also it would be useful to think about the impact of a completely wrong timestamp set by client. i.e. the robustness of the system.

Log Retention

There are both pros and cons for log retention to be based on CreateTime or LogAppendTime.

  • Pattern 1:
    Because the latency is small, so CreateTime and LogAppendTime will be close and their won't be much difference.
  • pattern 2:
    If the log retention is based on the message creation time, it will not be affected by the latency in the data pipeline because the send time will not change.
    If the log retention is based on the LogAppendTime, it will be affected by the latency in the pipeline. Because of the latency difference, some data can be deleted on one cluster, but not on another cluster in the pipeline.
  • Pattern 3: 
    When the messages with significantly different timestamp goes into a cluster at around same time, the retention policy is hard to follow if we use CreateTime. For example, imagine two mirror makers copy data from two source clusters to the same target cluster. If MirrorMaker1 is copying Messages with CreateTime around 1:00 PM today, and MirrorMaker2 is copying messages with CreateTime around 1:00 PM yesterday. Those messages can go to the same log segment in the target cluster. It will be difficult for broker to apply retention policy to the log segment. The broker needs to maintain the knowledge about the latest CreateTime of all messages in a log segment and persist the information somewhere.
  • Robustness: 
    If there is a message with CreateTime set to the future, the log might be kept for very long. Broker needs to sanity check the timestamp when receive the message. It could by tricky to determine which timestamp is not valid

Comparison:

 pattern 1pattern 2pattern 3Robustness
PreferenceCT or LATCTLATLAT

In reality, we usually don't see all the pipeline has same large latency, so it looks LogAppendTime is preferable than CreateTime for log retention.

Time based log rolling

The main purpose of time based log rolling is to avoid the situation where a low volume topic always has only one segment which is also the active segment. From its nature, server side log rolling makes more sense.

  • Pattern 1:
    Because the latency is small, so CreateTime and LogAppendTime will be close and their won't be much difference.
  • Pattern 2:
    When the latency is large, it is possible that when a new message is produced to a broker, the CreateTime has already reached the rolling criteria. This might cause a segment with only one message.
  • Pattern 3:
    Similar to pattern 2, a lagged message might result in a single message log segment.
  • Robustness:
    Similar to as pattern 2 and pattern 3. Also a CreateTime in the future might break the log rolling as well.

 

 pattern 1pattern 2pattern 3Robustness
PreferenceCT or LATLATLATLAT

Compatibility, Deprecation, and Migration Plan

NOTE: This part is drafted based on the assumption that KIP-31 and KIP-32 will be implemented in one patch.

The proposed protocol is not backward compatible. The migration plan are as below:

Phase 1 (MessageAndOffset V0 on disk):

  1. Set message.format.version=0 on brokers. (Broker will write MessageAndOffset V0 to disk)
  2. Create internal ApiVersion 0.9.0-1** which uses ProducerRequest V2 and FetchRequest V2.
  3. Configure the broker to use ApiVersion 0.9.0 (ProduceRequest V1 and FetchRequest V1).
  4. Do a rolling upgrade of the brokers to let the broker pick up the new code supporting ApiVersion 0.9.0-1.
  5. Bump up ApiVersion of broker to 0.9.0-1
  6. Do a rolling bounce of the brokers to let the broker use FetchRequest V2 for replication.
  7. Bump up ProducerRequest and FetchRequest version to V2, which supports both MessageAndOffset V0 and V1.
  8. Upgraded brokers support both ProducerRequest V2 and FetchRequest V2 which uses magic byte 1 for MessageAndOffset.
    1. When broker sees a producer request V1 (MessageAndOffset = V0), it will decompress the message, assign offsets using absolute offsets and NOT re-compress the message.
    2. When broker sees a producer request V2 (MessageAndOffset = V1), it will decompress the message, assign offsets using absolute offsets and do re-compression.  i.e. down-convert the message format to MessageAndOffset V0.
    3. When broker sees a fetch request V1 (Supporting MessageAndOffset = V0), because the data format on disk is MessageAndOffset V0, it will use the zero-copy transfer to reply with fetch response V1 with MessageAndOffset V0.
    4. When broker sees a fetch request V2 (Supporting MessageAndOffset = V0, V1), because the data format on disk is MessageAndOffset V0, it will use zero-copy transfer to reply with fetch response V2 with MessageAndOffset V0.
  9. Upgrade consumer to send FetchRequest V2.
  10. Upgrade producer to send ProducerRequest V2.

Phase 2 (MessageAndOffset V1 on disk):

  1. After most of the consumers are upgraded, Bump up message.format.version=1 and rolling bounce the brokers.
  2. Upgraded brokers do the followings:
    1. When broker sees a producer request V1 (MessageAndOffset = V0), it will decompress the message, assign offsets using relative offsets and re-compress the message. i.e. up-convert the message format to MessageAndOffset V1.
    2. When broker sees a producer request V2 (MessageAndOffset = V1), it will decompress the message, assign offsets using relative offsets and NOT do re-compression.
    3. When broker sees a fetch request V1 (Supporting MessageAndOffset = V0), because the data format on disk is MessageAndOffset V1, it will NOT use the zero-copy transfer. Instead the broker will read the message from disk, down-convert them to V0 and reply using fetch response V1 with MessageAndOffset V0.
    4. When broker sees a fetch request V2 (Supporting MessageAndOffset = V0, V1), because the data format on disk is MessageAndOffset V1, it will use zero-copy transfer to reply with fetch response V2 with MessageAndOffset V1.

For producer, there will be no impact.

In phase 1, there will be no impact for consumers.

In phase 2, there will be some performance penalty for consumers that only supports MessageAndOffset V0, because there is no zero-copy transfer.

At the beginning of phase 2, there will be some time the log segment contains both MessageAndOffset V0 and V1. The broker will always do down conversion for FetchRequest V1 and zero-copy transfer for FetchRequest V2.

** We introduce internal ApiVersion here to help the user who are running on trunk to upgrade in the future. Otherwise the interim ApiVersion between two official releases will require users to downgrade ApiVersion then upgrade.

To canary a broker

After phase 1, it is possible for user to canary a broker in phase 2 and roll back if something goes wrong. The procedure is:

  1. Set message.format.version=1 on one of the brokers (broker B).
  2. Broker B will start to act like what described in phase 2.
    1. It will sends FetchRequest V2 to other brokers for replication.
    2. It will only see ProduceRequest/FetchRequest V1 from other brokers and clietns.
  3. If something goes wrong, we can do the following to rollback:
    1. shutdown broker B
    2. nuke the data of the topics it was serving as leader before shutdown
    3. set message.format.version=0
    4. restart the broker to let the broker replicate from leaders. At this point the data on disk will be in MessageAndOffset V0.

In step 2, it is recommended to put only small amount of leaders on the broker, because at that point the broker needs to do down conversion for all the fetch requests.

 

 

Rejected Alternatives

Option 2 - Adding only LogAppendTime to the message

This proposal is pretty much the same as the selected proposal, except it does not include CreateTime in the message format.

MessageAndOffset => MessageSize Offset Message
  MessageSize => int32
  Offset => int64
  
  Message => Crc MagicByte Attributes Timestamp KeyLength Key ValueLength Value
    Crc => int32
    MagicByte => int8
    Attributes => int8
    LogAppendTime => int64 <---------------------- NEW
    KeyLength => int32
    Key => bytes
    ValueLength => int32
    Value => bytes

The downside of this proposal are:

  1. If the CreateTime is not in the message itself. Application needs to include the timestamp in payload. Instead of asking each application to do this, it is better to include the timestamp in the message.
  2. The broker is not able to report the latency metric. While we could let the application to get the End2End latency,  we might lose the latency for each hop in the pipeline.

 

  • No labels