Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Revised KIP so that consumer/follower have similar fencing behavior

...

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 problem in both of these cases is that the leader receiving the fetch request has no way to validate whether the fetcher's expectation about the current state of the log is correct. For the consumer, we need to check whether the log has been truncated since the previous fetch request. For the follower, we need to verify that the follower and the leader have matching epochs. 

In both cases, we need the Fetch request to include a field for the leader epoch. For the consumer, this epoch is the minimum expected epoch for the message at the fetch offset. The leader will use its leader epoch cache to validate that the epoch and fetch offset are correct. Specifically, it will ensure that there are no larger leader epochs which began at a smaller offset than the fetch offset. This would imply that the consumer did not see some messages from the later epoch, which means the log was truncated at some point. The leader will return a new LOG_TRUNCATION error code in this case. If the consumer has a larger epoch than the replica is aware of (the KIP-232 scenario), the leader will return UNKNOWN_LEADER_EPOCH.

The requirements for a following replica are slightly different. Upon becoming a follower, the replica has a truncation protocol which ensures that its log does not diverge from the leader. What is missing is proper fencing. A leader or follower change could be delayed indefinitely or even not received at all (as in the case of KAFKA-6880). The leader and follower must be able to ensure that they are on the same epoch for the replication protocol to work correctly.

So, for the replica, the leader epoch in the fetch request will be the current leader epoch. If the leader receives a fetch request from a replica with a lower epoch, it will return a FENCED_REPLICA error. As in the case of the consumer, if the epoch on the follower is larger than the leader is aware of, it will return UNKNOWN_LEADER_EPOCH.

In fact, the need to fence replicas extends to the truncation step in the become follower transition, so we propose to add the current leader epoch to the OffsetsForLeaderEpoch request as well. The epoch will be validated in the same way that we validate the epoch in the fetch request.

Below we elaborate on the changes to fetch handling in this KIP for replicas and consumers.

Leader Fetch Handling

When a broker receives a fetch request, it will validate the leader epoch. For replicas, the check is simply whether the epoch in the fetch request matches the current leader epoch. If the follower's epoch is smaller than the leader's, we return the new FENCED_REPLICA error code.

For the consumer, log truncation is detected if there exists a larger leader epoch with a starting offset which is smaller than the fetch offset. In this case, we know the log was truncated, so the leader will return a LOG_TRUNCATION error code in the fetch response.

Whether the fetch is from a replica or a consumer, if the epoch provided is larger than that of the leader itself, the broker will respond with a new UNKNOWN_LEADER_EPOCH error code which will cause the consumer to refresh metadata and try again. This is probably the result of stale metadata such as was observed in KIP-232

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

In order to handle the LOG_TRUNCATION error code, the consumer will need to find the offset at which its log diverged. We propose to use the OffsetForLeaderEpoch API that was introduced in KIP-101. As the consumer is fetching from a partition, it will keep track of the last epoch that was fetched for each partition. In the case of a log truncation error, it will use this epoch and the last fetched offset to find the point of divergence.

On the other hand, if the fetch returns an UNKNOWN_LEADER_EPOCH error code, then the consumer will refresh metadata and retry the fetch.

This KIP 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 is if the consumer fetches from an offset earlier than the log start offset. Otherwise, we will detect it either as a log truncation event or an instance of stale metadata (the KIP-232 scenario). 

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.  

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. To fix the problem with KIP-232, we will add the leader epoch the ListOffsets response. The consumer will use this in its first fetch request after resetting offsets. 
  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.
  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 new `seekToNearest()` API defined below.

Code Block
class LogTruncationException extends OffsetOutOfRangeException {}

We will also add a new retriable error code for the UNKNOWN_LEADER_EPOCH error code:

Code Block
class UnknownLeaderEpochException extends RetriableException {}

This will be located in the public `errors` package, but the consumer will internally retry when it receives this error.

The leader epoch will be exposed in the ConsumerRecord and OffsetAndMetadata objects.

Code Block
class ConsumerRecord<K, V> {
	int leaderEpoch();
}

class OffsetAndMetadata {
	int leaderEpoch();
}

We will also have a new API to support seeking to an offset and leader epoch. For convenience and consistency, we will reuse the OffsetAndMetadata object.

Code Block
/**
 * Seek to the nearest offset taking into account leader epoch information.
 * If the offset still exists and has the same leader epoch, then this will
 * set the position to that offset. Otherwise, if the log has been truncated
 * (e.g. due to unclean leader election), then the offset will be the first 
 * offset of the next largest leader epoch.
 */ 
void seekToNearest(TopicPartition partition, OffsetAndMetadata offset);

...

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. When a leader is changed, 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.

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 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.  

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. To fix the problem with KIP-232, we will add the leader epoch the ListOffsets response. The consumer will use this in its first fetch request after resetting offsets. 
  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.
  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.

Finally, we want to improve fencing for clients as well to resolve metadata inconsistencies such as in KIP-232. To support this, we will expose the leader epoch in the Metadata API. The consumer will use the current epoch when checking for truncation and when fencing.

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 new `seekToNearest()` API defined below.

Code Block
class LogTruncationException extends OffsetOutOfRangeException {}

We will also add a new retriable error code for the UNKNOWN_LEADER_EPOCH error code:

Code Block
class UnknownLeaderEpochException extends RetriableException {}

This will be located in the public `errors` package, but the consumer will internally retry when it receives this error.

The leader epoch will be exposed in the ConsumerRecord and OffsetAndMetadata objects.

Code Block
class ConsumerRecord<K, V> {
	int leaderEpoch();
}

class OffsetAndMetadata {
	int leaderEpoch();
}

We will also have a new API to support seeking to an offset and leader epoch. This is required in order to support storage of offsets in an external store. When the consumer is initialized, the user will call seekToCommitted with the offset and leader epoch that was stored. The consumer will initialize the position of the consumer using this API. If the log has been truncated since the time the offsets were stored, the next call to poll() will raise a LogTruncationException as described above.

Code Block
/**
 * Seek to a committed offset taking into account leader epoch information.
 */ 
void seekToCommitted(TopicPartition partition, OffsetAndMetadata offset);


Protocol Changes

Fetch

We will bump the fetch request version in order to include the last fetched leader epoch. This will only be provided by the fetcher for consecutive fetches. If the consumer seeks to the middle of the log, for example, then we will use the sentinel value -1 and the leader will skip the epoch validation. The replica should always have the previous epoch except when beginning from the beginning of the log.

The new schema is given below:

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
  		FetchOffset => INT64
  		StartOffset => INT64
		LeaderEpoch => INT32   // New
  		MaxBytes => INT32
  RemovedTopics => RemovedTopicName [RemovedPartition]
  	RemovedTopicName => STRING
  	RemovedPartition => INT32

The response schema will not change, but we will have two new error codes as mentioned above

  1. FENCED_LEADER_EPOCH: The replica has a lower epoch than the leader. This is retriable.
  2. UNKNOWN_LEADER_EPOCH: This is a retriable error code which indicates that the epoch in the fetch request was larger than any known by the broker.

OffsetsForLeaderEpoch

The changes to the OffsetsForLeaderEpoch request API are similar.

Code Block
linenumberstrue
OffsetForLeaderEpochRequest => [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.

Note that consumers will send a sentinel value (-1) for the current epoch and the broker will simply disregard that validation. 

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
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 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.

Fetch

We will bump the fetch request version in order to include the last fetched leader epoch. This will only be provided by the fetcher for consecutive fetches. If the consumer seeks to the middle of the log, for example, then we will use the sentinel value -1 and the leader will skip the epoch validation. The replica should always have the previous epoch except when beginning from the beginning of the log.

The new schema is given below:

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
  		FetchOffset => INT64
  		StartOffset => INT64
		LeaderEpoch => INT32   // New
  		MaxBytes => INT32
  RemovedTopics => RemovedTopicName [RemovedPartition]
  	RemovedTopicName => STRING
  	RemovedPartition => INT32

The response schema will not change, but we will have two new error codes as mentioned above

  1. LOG_TRUNCATION: This error code indicates that the last fetched offset and epoch do not match the local state of the log. This is only possible for consumers.
  2. FENCED_REPLICA: The replica has a lower epoch than the leader. This is retriable.
  3. UNKNOWN_LEADER_EPOCH: This is a retriable error code which indicates that the epoch in the fetch request was larger than any known by the broker.

OffsetsForLeaderEpoch

The changes to the OffsetsForLeaderEpoch request API are similar.

Code Block
linenumberstrue
OffsetForLeaderEpochRequest => [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.

Note that consumers will send a sentinel value (-1) for the current epoch and the broker will simply disregard that validation. 

OffsetCommit

The new OffsetCommit request schema is provided below. The response schema matches the previous version. We have added fields for the leader epoch and the timestamp.

...

  • When the consumer cannot find the precise truncation point, another option is to use the timestamp of the last consumed message in order to find the right offset to reset to. One benefit of this is that it works for older message formats. The downsides are 1) it complicates the API and 2) the truncation point determined using timestamp is not as accurate as what can be determined using the leader epoch. Ultimately we decided it was not worthwhile complicating the APIs for the old message format.
  • Initially, this proposal suggested that the follower should behave more like the consumer and include the expected next epoch in the fetch request rather than the current epoch. Unfortunately, model checking showed a weakness in this approach. To make the advancement of the high watermark safe, the leader must be be able to guarantee that all followers in the ISR are on the correct epoch. Any disagreement about the current leader epoch and the ISR can lead to the loss of committed data. See https://github.com/hachikuji/kafka-specification/blob/master/Kip320FirstTry.tla for more detail. We considered including both the current epoch and the expected epoch, but in neither case did we require both epochsthe truncation only occurs on leadership changes, so checking on every fetch was not necessary.