Versions Compared

Key

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

...

JIRA:

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-6880

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

Motivation

This KIP aims to solve two related problems in the handling of fetch requests.

Consumer handling of unclean truncation: When unclean leader election is enabled, we may lose committed data. A consumer which is reading from the end of the log will typically see an out of range error, which will cause it to use its auto.offset.reset policy. To avoid losing data, users should use the "earliest" option, but that means consuming the log from the beginning.

It is also possible that prior to sending the next fetch, new data is written to the log so that the consumer's fetch offset becomes valid again. In this case, the consumer will just miss whatever data had been written between the truncation point and its fetch offset.

Neither behavior is ideal, but we tend to overlook it because the user has opted into weaker semantics by enabling unclean leader election. Unfortunately in some situations we have to enable unclean leader election in order to recover from serious faults on the brokers. Some users have also opted to keep unclean leader election enabled because they cannot sacrifice availability ever. We would like to offer better client semantics for these situations.

Inadequate Replica Fencing: We encountered a situation in KAFKA-6880 in which a deadlock caused a broker to enter a zombie state in which it was no longer registered in zookeeper. Nevertheless, its fetchers continued attempting to make progress by fetching from leaders. Since it was no longer receiving updates from the controller, the leader metadata became stale. However, the replica was still able to make progress and even rejoin the ISR as the assigned leader periodically became accurate again.

The problem in this situation is that the replica, which is no longer registered in Zookeeper, does not receive any notifications from the controller about leader changes. We depend on these notifications so that the follower can use our log truncation protocol to find the right starting offset for the new leader. 

Proposed Changes

The lack of proper fencing has been a major weakness in Kafka, but the solution is straightforward since we already have the leader epoch to track leader changes. We will include the leader epoch in the Fetch request and the leader will reject the request if the epoch does not match its own. 

The problem for the consumer is that it does not expect to see truncation. It assumes data below the high watermark is never lost. This assumption is obviously violated in the case of unclean leader election. The solution we propose is to let the consumer behave more like a follower and check for truncation after observing a leader change.

Below we describe in more detail the behavior changes for leaders, followers, and consumers.

Leader Fetch Handling

This KIP adds the leader epoch to the Fetch request. When a broker receives a fetch request, it will compare the requested epoch with its own. The fetch will only be permitted if the requested epoch matches the leader's epoch. If the requested epoch is older than the leader's, we will use a new FENCED_LEADER_EPOCH error code. If the epoch is newer than the leader's (for example in a KIP-232 scenario), we will use a new UNKNOWN_LEADER_EPOCH error code.

For consumers, we do not require strict fencing of fetch requests. We will support a sentinel value for the leader epoch which can be used to bypass the epoch validation. Additionally, the truncation check is optional for consumers. A client can choose to skip it and retain the current semantics.

In addition to fencing the fetch request, we also need stronger fencing for the truncation phase of the transition to becoming a follower. Without it, we cannot be sure that the leader's log does not change between the time that the truncation occurs and the follower begins fetching. To support this, we will add the current leader epoch to the OffsetForLeaderEpoch API. The leader will validate it similarly to fetch requests. 

We will also change the leader behavior so that it is not permitted to add a replica to the ISR if it is marked as offline by the controller. By currently allowing this, we weaken acks=all semantics since the zombie contributes to the min.isr requirement, but is not actually eligible to become leader.

Consumer Handling

The proposal in this KIP is to have the consumer behave more like a follower. The consumer will obtain the current leader epoch using the Metadata API. When fetching from a new leader, the consumer will first check for truncation using the OffsetForLeaderEpoch API. In order to enable this, we need to keep track of the last epoch that was consumed. If we do not have one (e.g. because the user has seeked to a particular offset or because the message format is older), then the consumer will skip this step. To support this tracking, we will extend the OffsetCommit API to include the leader epoch if one is available.

Leader changes are detected either through a metadata refresh or in response to a FENCED_LEADER_EPOCH error. It is also possible that the consumer sees an UNKNOWN_LEADER_EPOCH in a fetch response if its metadata has gotten ahead of the leader. 

This change in behavior has implications for the consumer's offset reset policy, which defines what the consumer should do if its fetch offset becomes out of range. With this KIP, the only case in which this is possible other than an out of range seek is if the consumer fetches from an offset earlier than the log start offset. By opting into an offset reset policy, the user allows for automatic adjustments to the fetch position, so we take advantage of this to to reset the offset as precisely as possible when log truncation is detected. In some pathological cases (e.g. multiple consecutive unclean leader elections), we may not be able to find the exact offset, but we should be able to get close by finding the starting offset of the next largest epoch that the leader is aware of. We propose in this KIP to change the behavior for both the "earliest" and "latest" reset modes to do this automatically as long as the message format supports lookup by leader epoch. The consumer will log a message to indicate that the truncation was detected, but will reset the position automatically. 

If a user is not using an auto reset option, we will raise a LogTruncationException from poll() when log truncation is detected. This gives users the ability to reset state if needed or revert changes to downstream systems. The exception will include the partitions that were truncated and the offset of divergence as found above. This gives applications the ability to execute any logic to revert changes if needed and rewind the fetch offset. Users must handle this exception and reset the position of the consumer if no auto reset policy is enabled.

For consumers, we propose some additional extensions:

  1. When the consumer needs to reset offsets, it uses the ListOffsets API to query the leader. To avoid querying stale leaders, we will add epoch fencing. Additionally, we will modify this API to return the corresponding epoch for any offsets looked up.
  2. We will also provide the leader epoch in the offset commit and fetch APIs. This allows consumer groups to detect truncation across rebalances or restarts. Note that in cases like that found in KIP-232, it is possible for the leader epoch included in the committed offset to be ahead of the metadata that is known to the consumer. Consumers are expected to wait until the metadata has at least reached the epoch of the committed offset before checking for truncation.
  3. For users that store offsets in an external system, we will provide APIs which expose the leader epoch of each record and we will provide an alternative seek API so that users can initialize the offset and leader epoch.

Replica Handling

This KIP adds replica fencing for both fetching and truncation. When a replica becomes a follower, it will attempt to find the truncation offset using the OffsetsForLeaderEpoch API. The new epoch will be included in this request so that the follower is ensured that it is truncating using the log from the leader in the correct epoch. Without this validation, it is possible to truncate using the log of a leader with stale log state, which can lead to log divergence.

After the follower has truncated its log, it will begin fetching as it does today. It will similarly include the current leader epoch, which will be validated by the leader. If the fetch response contains either the FENCED_REPLICA or UNKNOWN_LEADER_EPOCH error code, the follower will simply retry since both errors may be transient (e.g. if the propagation of the LeaderAndIsr request is delayed). 

Note that we have implemented the this fetching model in TLA. So far, we have not found any errors checking this model.

Public Interfaces

API Changes

We will introduce a new exception type, which will be raised from KafkaConsumer.poll(Duration) as described above. This exception extends from `OffsetOutOfRangeException` for compatibility. If a user's offset reset policy is set to "none," they will still be able to catch `OffsetOutOfRangeException`. For new usage, users can catch the more specific exception type and use the `seek()` API to resume consumption. This exception is raised by the consumer after using the OffsetForLeaderEpoch API to find the offset of divergence. This offset is included as a field in the exception. Typically users handling this exception will seek to this offset.

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-7395
Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-7440
,  
Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-7747

Release: The broker side changes which improved fencing were released in 2.1.0. Client-side truncation detection and reset capability was released in 2.3.0

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

Motivation

This KIP aims to solve two related problems in the handling of fetch requests.

Consumer handling of unclean truncation: When unclean leader election is enabled, we may lose committed data. A consumer which is reading from the end of the log will typically see an out of range error, which will cause it to use its auto.offset.reset policy. To avoid losing data, users should use the "earliest" option, but that means consuming the log from the beginning.

It is also possible that prior to sending the next fetch, new data is written to the log so that the consumer's fetch offset becomes valid again. In this case, the consumer will just miss whatever data had been written between the truncation point and its fetch offset.

Neither behavior is ideal, but we tend to overlook it because the user has opted into weaker semantics by enabling unclean leader election. Unfortunately in some situations we have to enable unclean leader election in order to recover from serious faults on the brokers. Some users have also opted to keep unclean leader election enabled because they cannot sacrifice availability ever. We would like to offer better client semantics for these situations.

Inadequate Replica Fencing: We encountered a situation in KAFKA-6880 in which a deadlock caused a broker to enter a zombie state in which it was no longer registered in zookeeper. Nevertheless, its fetchers continued attempting to make progress by fetching from leaders. Since it was no longer receiving updates from the controller, the leader metadata became stale. However, the replica was still able to make progress and even rejoin the ISR as the assigned leader periodically became accurate again.

The problem in this situation is that the replica, which is no longer registered in Zookeeper, does not receive any notifications from the controller about leader changes. We depend on these notifications so that the follower can use our log truncation protocol to find the right starting offset for the new leader. 

Proposed Changes

The lack of proper fencing has been a major weakness in Kafka, but the solution is straightforward since we already have the leader epoch to track leader changes. We will include the leader epoch in the Fetch request and the leader will reject the request if the epoch does not match its own. 

The problem for the consumer is that it does not expect to see truncation. It assumes data below the high watermark is never lost. This assumption is obviously violated in the case of unclean leader election. The solution we propose is to let the consumer behave more like a follower and check for truncation after observing a leader change.

Below we describe in more detail the behavior changes for leaders, followers, and consumers.

Leader Fetch Handling

This KIP adds the leader epoch to the Fetch request. When a broker receives a fetch request, it will compare the requested epoch with its own. The fetch will only be permitted if the requested epoch matches the leader's epoch. If the requested epoch is older than the leader's, we will use a new FENCED_LEADER_EPOCH error code. If the epoch is newer than the leader's (for example in a KIP-232 scenario), we will use a new UNKNOWN_LEADER_EPOCH error code.

For consumers, we do not require strict fencing of fetch requests. We will support a sentinel value for the leader epoch which can be used to bypass the epoch validation. Additionally, the truncation check is optional for consumers. A client can choose to skip it and retain the current semantics.

In addition to fencing the fetch request, we also need stronger fencing for the truncation phase of the transition to becoming a follower. Without it, we cannot be sure that the leader's log does not change between the time that the truncation occurs and the follower begins fetching. To support this, we will add the current leader epoch to the OffsetForLeaderEpoch API. The leader will validate it similarly to fetch requests. 

We will also change the leader behavior so that it is not permitted to add a replica to the ISR if it is marked as offline by the controller. By currently allowing this, we weaken acks=all semantics since the zombie contributes to the min.isr requirement, but is not actually eligible to become leader.

Consumer Handling

The proposal in this KIP is to have the consumer behave more like a follower. The consumer will obtain the current leader epoch using the Metadata API. When fetching from a new leader, the consumer will first check for truncation using the OffsetForLeaderEpoch API. In order to enable this, we need to keep track of the last epoch that was consumed. If we do not have one (e.g. because the user has seeked to a particular offset or because the message format is older), then the consumer will skip this step. To support this tracking, we will extend the OffsetCommit API to include the leader epoch if one is available.

Leader changes are detected either through a metadata refresh or in response to a FENCED_LEADER_EPOCH error. It is also possible that the consumer sees an UNKNOWN_LEADER_EPOCH in a fetch response if its metadata has gotten ahead of the leader. 

This change in behavior has implications for the consumer's offset reset policy, which defines what the consumer should do if its fetch offset becomes out of range. With this KIP, the only case in which this is possible other than an out of range seek is if the consumer fetches from an offset earlier than the log start offset. By opting into an offset reset policy, the user allows for automatic adjustments to the fetch position, so we take advantage of this to to reset the offset as precisely as possible when log truncation is detected. In some pathological cases (e.g. multiple consecutive unclean leader elections), we may not be able to find the exact offset, but we should be able to get close by finding the starting offset of the next largest epoch that the leader is aware of. We propose in this KIP to change the behavior for both the "earliest" and "latest" reset modes to do this automatically as long as the message format supports lookup by leader epoch. The consumer will log a message to indicate that the truncation was detected, but will reset the position automatically. 

If a user is not using an auto reset option, we will raise a LogTruncationException from poll() when log truncation is detected. This gives users the ability to reset state if needed or revert changes to downstream systems. The exception will include the partitions that were truncated and the offset of divergence as found above. This gives applications the ability to execute any logic to revert changes if needed and rewind the fetch offset. Users must handle this exception and reset the position of the consumer if no auto reset policy is enabled.

For consumers, we propose some additional extensions:

  1. When the consumer needs to reset offsets, it uses the ListOffsets API to query the leader. To avoid querying stale leaders, we will add epoch fencing. Additionally, we will modify this API to return the corresponding epoch for any offsets looked up.
  2. We will also provide the leader epoch in the offset commit and fetch APIs. This allows consumer groups to detect truncation across rebalances or restarts. Note that in cases like that found in KIP-232, it is possible for the leader epoch included in the committed offset to be ahead of the metadata that is known to the consumer. Consumers are expected to wait until the metadata has at least reached the epoch of the committed offset before checking for truncation.
  3. For users that store offsets in an external system, we will provide APIs which expose the leader epoch of each record and we will provide an alternative seek API so that users can initialize the offset and leader epoch.

Replica Handling

This KIP adds replica fencing for both fetching and truncation. When a replica becomes a follower, it will attempt to find the truncation offset using the OffsetsForLeaderEpoch API. The new epoch will be included in this request so that the follower is ensured that it is truncating using the log from the leader in the correct epoch. Without this validation, it is possible to truncate using the log of a leader with stale log state, which can lead to log divergence.

After the follower has truncated its log, it will begin fetching as it does today. It will similarly include the current leader epoch, which will be validated by the leader. If the fetch response contains either the FENCED_REPLICA or UNKNOWN_LEADER_EPOCH error code, the follower will simply retry since both errors may be transient (e.g. if the propagation of the LeaderAndIsr request is delayed). 

Note that we have implemented the this fetching model in TLA. So far, we have not found any errors checking this model.

Public Interfaces

API Changes

We will introduce a new exception type, which will be raised from KafkaConsumer.poll(Duration) as described above. This exception extends from `OffsetOutOfRangeException` for compatibility. If a user's offset reset policy is set to "none," they will still be able to catch `OffsetOutOfRangeException`. For new usage, users can catch the more specific exception type and use the `seek()` API to resume consumption. This exception is raised by the consumer after using the OffsetForLeaderEpoch API to find the offset of divergence. This offset is included as a field in the exception. Typically users handling this exception will seek to this offset.

Code Block
/**
 * In the even of unclean leader election, the log will be truncated,
 * previously committed data will be lost, and new data will be written
 * over these offsets. When this happens, 
Code Block
/**
 * In the even of unclean leader election, the log will be truncated,
 * previously committed data will be lost, and new data will be written
 * over these offsets. When this happens, the consumer will detect the 
 * truncation and raise this exception (if no automatic reset policy 
 * has been defined) with the first offset to diverge from what the 
 * consumer read.
 */ 
class LogTruncationException extends OffsetOutOfRangeException {
  /**
   * Get the truncation offsets for the partitions which were truncated. 
   * This is the first offset which is known to diverge from what the consumer read.
   */	
  Map<TopicPartition, OffsetAndMetadata> truncationOffsets();
}

...

Code Block
linenumberstrue
FetchRequest => MaxWaitTime ReplicaId MinBytes IsolationLevel FetchSessionId FetchSessionEpoch [Topics] [RemovedTopics]
  MaxWaitTime => INT32
  ReplicaId => INT32
  MinBytes => INT32
  IsolationLevel => INT8
  FetchSessionId => INT32
  FetchSessionEpoch => INT32
  Topics => TopicName Partitions
  	TopicName => STRING
  	Partitions => [Partition FetchOffset StartOffset LeaderEpoch MaxBytes]
  		Partition => INT32
  		FetchOffsetCurrentLeaderEpoch => INT64 INT32   // New
  		StartOffsetFetchOffset => INT64
  		CurrentLeaderEpochStartOffset => INT32   // NewINT64
  		MaxBytes => INT32
  RemovedTopics => RemovedTopicName [RemovedPartition]
  	RemovedTopicName => STRING
  	RemovedPartition => INT32

...

Code Block
linenumberstrue
OffsetForLeaderEpochRequestOffsetsForLeaderEpochRequest => [Topic]
  Topic => TopicName [Partition] 
    TopicName => STRING
    Partition => PartitionId CurrentLeaderEpoch LeaderEpoch
      PartitionId => INT32
      CurrentLeaderEpoch => INT32 // New
      LeaderEpoch => INT32

If the current leader epoch does not match that of the leader, then we will send either FENCED_REPLICA or UNKNOWN_LEADER_EPOCH as we do for the Fetch API.

Metadata

The metadata response will be extended to include the leader epoch. This enables stronger fencing for consumers. We can enable similar protection in producers in the future, but that is out of the scope of this KIP.

Since this API is no longer exclusively an inter-broker API, we will also add the throttle time to the response schema.

Code Block
linenumberstrue
OffsetsForLeaderEpochResponse => ThrottleTimeMs [TopicMetadata]
  ThrottleTimeMs => INT32 // New
  TopicMetadata => TopicName PartitionMetadata
    TopicName => STRING
    PartitionMetadata => [ErrorCode PartitionId LeaderEpoch EndOffset]
      ErrorCode => INT16
      PartitionId => INT32
      LeaderEpoch => INT32
      EndOffset => INT64


Metadata

The metadata response will be extended to include the leader epoch. This enables stronger fencing for consumers. We can enable similar protection in producers in the future, but that is out of the scope of this KIP.

Code Block
linenumberstrue
MetadataResponse => ThrottleTimeMs Brokers ClusterId ControllerId [TopicMetadata]
  ThrottleTimeMs => INT32
  Brokers => [MetadataBroker]
  ClusterId => NULLABLE_STRING
  ControllerId => INT32
  
TopicMetadata => ErrorCode TopicName IsInternal [PartitionMetadata]
  ErrorCode => INT16
  TopicName => STRING
  IsInternal => BOOLEAN
  
PartitionMetadata => ErrorCode PartitionId Leader LeaderEpoch Replicas ISR OfflineReplicas
  ErrorCode => INT16
  PartitionId => INT32
  Leader => INT32
  LeaderEpoch => INT32 // New
  Replicas => [INT32]
  ISR => [INT32]
  OfflineReplicas => [INT32]

There are no changes to the Metadata request schema.

OffsetCommit

The new OffsetCommit request schema is provided below. A field for the leader epoch has been added. Note that this is the epoch of the previously fetched record. The response schema matches the previous version.

Code Block
linenumberstrue
OffsetCommitRequest => GroupId Generation MemberId [TopicName [Partition Offset LeaderEpoch Metadata]]
  GroupId
Code Block
linenumberstrue
MetadataResponse => Brokers ClusterId ControllerId [TopicMetadata]
  Brokers => [MetadataBroker]
  ClusterId => NULLABLE_STRING
  ControllerId => INT32
  
TopicMetadata => ErrorCode TopicName IsInternal [PartitionMetadata]
  ErrorCode => INT16
  TopicName => STRING
  IsInternal => BOOLEAN
  
PartitionMetadata => ErrorCode PartitionId Leader LeaderEpoch Replicas ISR OfflineReplicasGeneration => INT32
  ErrorCodeMemberId => INT16STRING
  PartitionIdRetentionTime => INT32INT64
  LeaderTopicName => INT32STRING
  LeaderEpochPartition => INT32 // New
  ReplicasOffset => [INT32]INT64
  ISRLeaderEpoch => [INT32]INT32  // New
  OfflineReplicasMetadata => [INT32]

There are no changes to the Metadata request schema.

OffsetCommit

STRING

TxnOffsetCommit

Similarly, we need to add the leader epoch to the TxnOffsetCommit API, which is used by transactional producers.The new OffsetCommit request schema is provided below. A field for the leader epoch has been added. Note that this is the epoch of the previously fetched record. The response schema matches the previous version.

Code Block
linenumberstrue
OffsetCommitRequestTxnOffsetCommitRequest => TransactionalId GroupId ProducerId ProducerEpoch Topics
 Generation MemberIdGroupId [TopicName [Partition Offset Timestamp LastLeaderEpoch Metadata]]=> STRING
  Generation => INT32
  GroupIdMemberId => STRING
  GenerationRetentionTime => INT32INT64
  MemberIdTopics => STRING
  RetentionTime => INT64 [TopicName [Partition Offset LeaderEpoch Metadata]]  
    TopicName => STRING
    Partition => INT32
    Offset => INT64
    LeaderEpoch => INT32  // New
    Metadata => STRING


OffsetFetch

...

Code Block
linenumberstrue
OffsetFetchResponse => ThrottleTimeMs Topics ErrorCode
  ThrottleTimeMs => INT64
  Topics => [TopicName [Partition Offset Timestamp LastLeaderEpochLeaderEpoch Metadata ErrorCode]]
  	TopicName => STRING
  	Partition => INT32
  	Offset => INT64
  	LeaderEpoch => INT32  // New
  	Metadata => STRING
    MetadataErrorCode => STRINGINT16
  ErrorCode => INT16

ListOffsets

...

Code Block
linenumberstrue
ListOffsetRequest => ReplicaId [TopicName [Partition CurrentLeaderEpoch TimeTimestamp]]
 ReplicaId => INT32
 TopicName => STRING
 Partition => INT32
 CurrentLeaderEpoch => INT32 // New
 TimeTimestamp => INT64

Second, we add the leader epoch to the response that corresponds with the returned offset. The client can use this to reconcile the log prior to fetching from a new leader.

Code Block
linenumberstrue
ListOffsetResponse => ThrottleTimeMs Topics
  ThrottleTimeMs => INT64
  Topics => [TopicName [Partition ErrorCode Timestamp Offset LeaderEpoch Timestamp ErrorCode]]
    TopicName => STRING
    Partition => INT32
  Offset  ErrorCode => INT64INT16
  LeaderEpoch  Timestamp => INT32INT64   // New
  Timestamp  Offset => INT64
    ErrorCodeLeaderEpoch => INT16INT32  // New

As with the Fetch and OffsetForLeaderEpoch APIs, the response will support the FENCED_LEADER_EPOCH and UNKNOWN_LEADER_EPOCH error codes.

...