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

Compare with Current View Page History

« Previous Version 4 Next »

Status

Current stateUnder Discussion

Discussion thread: TBD

JIRA: Unable to render Jira issues macro, execution error.

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

Motivation

Transactional coordinator uses producer epoch to guarantee a single writer scope for each transactional id. When a new producer with the same transactional id starts up and talks to the coordinator, the coordinator will bump the epoch. Thus when the older producer attempts to make progress, it shall be fenced by a fatal `ProducerFenced` exception as it still uses old epoch.

However, producer epoch could also be bumped when a transaction times out on the coordinator side. This could be caused by a short period of inactivity of the client due to network partition or long GC. When the producer goes back online and attempts to proceed, it will receive the exact `ProducerFenced` even though a conflicting producer doesn't exist. The application has to shut down the current producer and rebuild a new one instead, by placing an extra try-catch logic which is cumbersome.

We can improve this with the new APIs from KIP-360. When the coordinator times out a transaction, it can remember that fact and allow the existing producer to claim the bumped epoch and continue.

Public Interfaces

We will add a retriable error code to allow producer distinguish a fatal fencing vs a soft retry after server side timeout:

TRANSACTION_TIMED_OUT(90, "The last ongoing transaction timed out on the coordinator, should retry initialization with current epoch", TransactionTimedOutException::new);

Proposed Changes

The workflow shall look like:

1. When a transaction times out, set lastProducerEpoch to the current epoch and do the normal bump.

2. Any transactional requests from the old epoch result in a new TRANSACTION_TIMED_OUT error code, which is propagated to the application.

3. The producer recovers by sending InitProducerId with the current epoch. The coordinator returns the bumped epoch.

One extra issue that needs to be addressed is how to handle INVALID_PRODUCER_EPOCH from Produce requests. Partition leaders will not generally know if a bumped epoch was the result of a timed out transaction or a fenced producer. In this case, new producers can treat INVALID_PRODUCER_EPOCH as abortable when they come from Produce responses. Consequently Producer would try to abort the transaction to detect whether this was due to a timeout or otherwise, as end transaction call shall also be protected by the new transaction timeout retry logic.

Compatibility, Deprecation, and Migration Plan

This change shall only apply to new Producer clients and new brokers, thus having no impact on existing user behavior. To be able to benefit from this feature, user only needs to upgrade both clients and brokers to latest version, without ordering requirement.

Rejected Alternatives

For the INVALID_PRODUCER_EPOCH, we could also consider keeping the current behavior still and let user do the abort transactions manually when catching the exception. This seems to be unnecessary and has more cost to educate users, thus we propose to let Producer handle this scenario internally.


  • No labels