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

Compare with Current View Page History

« Previous Version 2 Next »

Status

Current state: "Draft"

Discussion thread: here [Change the link from the KIP proposal email archive to your own email thread]

JIRA: here [Change the link from KAFKA-1 to your own ticket]

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

Background

2PC Refresher

Two phase commit (a.k.a. 2PC) protocol is used to implement distributed transactions across multiple participants.  Each participant manages its own state that is not recovered from other participants (if some datasets can be fully recovered from another source they are considered as one participant in 2PC protocol, e.g. a database could have multiple tables and indices, but they are all recovered from one log, so it’s one participant; on the other hand a broker manages multiple partitions but they don’t recover from each other, so they are multiple participants).  One of the participants (or a separate entity) is designated to be a transaction coordinator.  A transaction coordinator could be different for each transaction (e.g. the first participant of a transaction becomes the transaction coordinator) or it could be the same (e.g. if we use a designated transaction coordinator it’s generally used for all transactions); in any case the role of the transaction coordinator for a transaction is that it stores the decision for the transaction outcome (commit or abort) and is able to eventually deliver it to all transaction participants.

The 2PC protocol is the following:

  1. Each participant does some atomic transformations.  It is generally assumed that each participant can support non-distributed transactions, in which case no additional logic is required by the 2PC: a failed transaction aborts and gets back to a state before the transaction started.  If any participant fails while doing some atomic transformation, the distributed transaction is aborted.
  2. When a distributed transaction is ready to commit, all participants are asked to prepare the transaction to commit.  This is the PREPARE phase (the first phase of 2PC).  Each participant is free to say ‘yes’ or ‘no’; if any participant says ‘no’ (or just fails or times out or etc.) the transaction is aborted.  If a participant says ‘yes’ during the PREPARE phase, it must be able to commit or abort at will, based on the coordinator’s decision and must remain in-doubt until it receives the coordinator’s decision across all failures of the participant.  A transaction that is prepared in a participant is said to be in an in-doubt state (or just in-doubt transaction) – in this state neither the new nor the old state can be revealed, locks are held, etc.  All failure modes should be eliminated – data must be flushed and cannot be lost on restart, log needs to have space to store the commit marker, etc. – a commit cannot fail.
  3. Once all the participants are prepared, we can make a transaction outcome decision.  It could be either commit or abort.  If any of the participants replied ‘no’ during the PREPARE phase the transaction would be aborted.  If all participants reply ‘yes’ then the transaction coordinator tries to commit the transaction (e.g. by writing a record to its log).  If the commit operation succeeds in the transaction coordinator, then the transaction becomes committed.  This is the COMMIT phase, the 2nd phase of the 2PC protocol.  The decision is communicated to all participants eventually.  The fact that some participants may receive the outcome decision later than the other doesn’t affect correctness, because a participant that hasn't received the decision yet doesn’t reveal the outcome (locks are held, etc.).

There are variations of implementations (e.g. how the commit decision is communicated – could be pushed by the transaction coordinator, pulled by participants, “gossiped” by the client [if the client saw a commit of a transaction on one participant other participants can infer that it’s committed], etc.).

Kafka as a Participant in 2PC

Kafka is a distributed system, so transactions in Kafka are already distributed internally.  The 2PC maps to Kafka protocol as the following:

  1. Each partition is registered with the Kafka transaction coordinator so that it would abort the transaction in case of abort.  Messages are produced to each partition as needed.
  2. When a transaction is ready to commit, data is flushed to all data partitions (and commit offsets are flushed to the corresponding offset_commit partitions).  If any flush fails, then it’s treated as ‘no’ answer and the transaction is aborted.  If all flushes are successful, then the commit decision is sent to the Kafka transaction coordinator to be recorded.  The “prepared” transaction data is not visible to consumers reading with ‘read_committed’ isolation level.
  3. The Kafka transaction coordinator writes a PREPARE_COMMIT (or PREPARE_ABORT) record to its log to record a transaction outcome decision.  The name is somewhat misleading, because this is the COMMIT phase and once the PREPARE_COMMIT record is written, the transaction is committed and it’s just a matter of making the transaction participants aware of decisions.  The Kafka transaction coordinator pushes transaction outcome decisions until it’s propagated to all participating partitions and wouldn’t let another transaction (with the same transactional.id) started, until the transaction decision is delivered to all participating partitions.

To protect from client failures and avoid transactions to be in-doubt for a long time, Kafka has a limit on how long a transaction can be running - transaction.max.timeout.ms (15 minutes by default), so if a transaction is running longer than the limit it’s going to be aborted.

From that perspective, Kafka already has a 2PC implementation, but it’s controlled by Kafka transaction coordinator.  If we could support an external transaction coordinator, Kafka would be able to become a participant in a distributed transaction with other systems.

The main requirement for a participant is that once prepared it cannot abort (or commit) on its own and must wait for a decision made by the external transaction coordinator.  Currently, Kafka violates this requirement in two cases:

  1. On client restart, the client must use InitProducerId RPC to start using transactions.  InitProducerId aborts the on-going transaction, so even though the transaction is prepared (all data is flushed in place) it cannot survive client restart.
  2. A transaction that’s running longer than transaction.max.timeout.ms would be aborted (thus would make a decision that may be different from the external transaction coordinator’s decision).

Fixing these cases would let Kafka become a participant in an externally coordinated distributed transaction.

Motivation

One important use case that currently has no good solution is supporting Exactly Once Semantics (EOS) between Flink’s KafkaSink operator and Kafka.  The KafkaSink operator uses Kafka transactions to implement EOS across Flink and Kafka.  In that case, Flink’s job manager effectively acts as an external transaction coordinator in a 2PC protocol and Kafka is one of the participants.  For more details about the Flink KafkaSink operation and how it could utilize Kafka 2PC see FLIP-319.

The KafkaSink operator manages to work around the first problem (InitProducerId aborting the transaction) using reflection: KafkaSink keeps track of the producerId and epoch, so if it needs to commit a transaction after the producer has crashed, it could just issue a commit without going through the InitProducerId (which would abort the transaction).  Obviously, using reflection leads to maintenance nightmare so adding official support into Kafka so that KafkaSink could use a public API instead of reflection is a great improvement.

The second problem (transaction timeout) cannot be robustly solved without support from Kafka.  The current workaround is to crank up transaction.max.timeout.ms and hope that Flink won’t hold a transaction for a long period of time, but “hoping” is not a very robust way to provide technical guarantees.

Scope

The scope of this KIP is to enable Kafka to be a participant in a 2PC protocol.  It is expected that the transaction coordination (which includes proper tracking of transaction state) is done by an external system (Flink, connectors, etc.).  Any additional distributed transaction functionality (e.g. using Kafka as a transaction coordinator, supporting Open XA) is out of the scope of this KIP.

Solution Requirements and Constraints

In the essence we need two changes:

  1. Officially support functionality that Flink uses anyway via reflection.
  2. Relax / remove the timeout for transactions participating in 2PC.

Adding support for 1 should have no technical implications, we just publicly support functionality that’s currently used via reflection anyway.

Adding support for 2 requires considering why we have timeout in the first place and how we can mitigate problems that could arise if we remove timeout.

When a partition has an ongoing transaction, there are some implications: consumers that use read_committed isolation level cannot consume past the ongoing transaction (even if there are committed transactions later in the partition, consumers with read_committed isolations won’t read past messages that are part of an ongoing transaction), compaction wouldn’t compact past ongoing transactions.

transaction.max.timeout.ms guarantees that an ongoing transaction is aborted within a reasonable amount of time, but to avoid violation of the 2PC protocol we need to keep the transaction open, which could put pressure on the system.  To mitigate this impact, we should restrict the ability to run 2PC protocol via a privilege, so that it’s easy to protect the cluster from a random rogue application.

Even with restricting 2PC functionality to well behaving applications, we cannot prevent cases of long running and / or abandoned transactions.  We need some metrics to detect long running transactions.  We also need a way for the admin to find and forcibly abort long running and abandoned transactions in case the application cannot properly complete them.

Proposed Changes

The InitProducerId API would provide way to indicate that:

  • This transactional producer is a participant in 2PC protocol
  • Don’t abort the ongoing transaction (so that it’s possible to commit a prepared transaction even after producer restarts)

The broker will check if the client is allowed to use 2PC protocol, the request will fail if 2PC protocol is not allowed for the client.

If the InitProducerId was asked to not abort the ongoing transaction, then the application is only allowed to call .commitTransaction or .abortTransaction, calling other methods (e.g. .send) would fail.  This is because the reason to call InitProducerId without aborting the ongoing transaction is to complete a prepared transaction after producer’s crash.

If the transaction is a participant of the 2PC protocol, we don’t limit the transactional timeout by the transaction.max.timeout.ms any more, so if the client passes max integer value, the broker can accept it.

The admin client would support a new method to abort a transaction with a given transactional id.  The method would just execute InitProducerId.

A new metric would be added to track ongoing transaction time.

Public Interfaces

RPC Changes

We will bump the InitProducerId API.  The new request schema is going to be the following:

InitProducerIdRequest => TransactionalId TransactionTimeoutMs ProducerId Epoch Enable2Pc KeepPreparedTxn
 TransactionalId => NULLABLE_STRING
 TransactionTimeoutMs => INT32
 ProducerId => INT64
 Epoch => INT16
 Enable2Pc => BOOL  // NEW
 KeepPreparedTxn => BOOL  // NEW

We’ll also bump ListTransactionsRequest to support listing only transactions that are running longer than a certain amount of time.

ListTransactionsRequest => StatesFilter ProducerIdFilter RunningLongerThanMs
 StatesFilter => []STRING
 ProducerIdFilter => []STRING
 RunningLongerThanMs => INT64 // NEW

Metric Changes

A new metric will be added

active-transaction-total-time-max The max time a currently-open transaction has been open

To calculate the metrics we’ll just walk through the ongoing transactions and record the max value.

Client Configuration Changes

transaction.two.phase.commit.enable The default would be ‘false’.  If set to ‘true’, then the broker is informed that the client is participating in two phase commit protocol and can set transaction timeout to values that exceed transaction.max.timeout.ms setting on the broker (if the timeout is not set explicitly on the client and the two phase commit is set to ‘true’ then the transaction never expires).

transaction.timeout.ms The semantics is not changed, but it can be set to values that exceed transaction.max.timeout.ms if two.phase.commit.enable is set to ‘true’.

Broker Configuration Changes

transaction.two.phase.commit.enable The default would be ‘false’.  If it’s ‘false’, 2PC functionality is disabled even if the ACL is set, clients that attempt to use this functionality would receive TRANSACTIONAL_ID_AUTHORIZATION_FAILED error.

KafkaProducer API Changes

New overloaded method will be added to KafkaProducer:

public void initTransactions(boolean keepPreparedTxn) 

If the value is 'true' then the corresponding field is set in the InitProducerIdRequest and the KafkaProducer object is set into a state which only allows calling .commitTransaction or .abortTransaction.

If the transaction.two.phase.commit.enable setting is set to ‘false’ but keepPreparedTxn is set to ‘true’ then the call fails with the INVALID_TXN_STATE error.

New method will be added to KafkaProducer:

public void prepareTransaction() 

This would flush all the pending messages and transition the producer into a mode where only .commitTransaction  or .abortTransaction  could be called.  If the call is successful (all messages successfully got flushed to all partitions) the transaction is prepared.  If the 2PC is not enabled, we return the INVALID_TXN_STATE error.

AdminClient API Changes

Class ListTransactionsOptions will support 2 new methods:

public ListTransactionsOptions runningLongerThanMs(long runningLongerThanMs) // set 

public long runningLongerThanMs() // get 

The Admin  interface will support a new method:

public TerminateTransactionResult forceTerminateTransaction(String transactionalId) 

TerminateTransactionResult just contains KafkaFuture<void> result method.

NOTE that there is an existing abortTransaction  method that is used to abort “hanging” transactions (artifact of some gaps in the transaction protocol implementation that will be addressed in KIP-890, i.e. once part 1 of KIP-890 is implemented we won’t have “hanging” transactions).  “Hanging” transactions are not known to the Kafka transaction coordinator, they are just dangling messages in data partitions that cannot be aborted via the normal transaction protocol.  So abortTransaction actually needs information about data partitions so that it could go and insert markers directly there.

On the other hand, the forceTerminateTransaction method would operate on a well-formed, but long running transaction for a given transactional id.  Under the covers it would just use InitProducerId call with keepPreparedTxn=false.

ACL Changes

A new value will be added to the enum AclOperation: TWO_PHASE_COMMIT ((byte) 15 .  When InitProducerId comes with enable2Pc=true, it would have to have both WRITE and TWO_PHASE_COMMIT operation enabled on the transactional id resource.

Command Line Tool Changes

[TODO] spell out changes, but basically mostly mechanical changes to expose the corresponding admin API changes:

  • New filter for listing transactions
  • New command to force-terminate transactions
  • New ACL

Compatibility, Deprecation, and Migration Plan

The proposal doesn't remove or update any existing functionality, it just adds new functionality that would only be executed if the new configurations and APIs are used.

Test Plan

The corresponding unit an integration tests will be added.

Rejected Alternatives

Explicit “prepare” RPC

Given that the 2PC protocol is defined in terms of “prepare” and “commit” phases it seems natural to just add a “prepare” RPC to Kafka.  The RPC would tell the Kafka transaction coordinator to transition the transaction into a new “prepared” state (note that our current state names are misleading – PREPARE_COMMIT is actually the “commit” phase).

There are some potential benefits of doing that:

  1. Transactions that haven’t reached “prepared” state can be aborted via timeout.
  2. New updates to “prepared” transactions can be rejected.
  3. InitProducerId would know to not abort prepared transactions.
  4. We could query for prepared transactions.

The disadvantage of an explicit “prepare” state is that we’d need to run a synchronous operation on the Kafka transaction coordinator topic and (if we want to support the benefit 2) send “prepare” markers to all leaders (so that they could bounce off new updates).

At a closer examination the benefits are not really eliminating any complexities:

  1. We still need tooling and operational support to handle transactions that are stuck in the “prepared” state.
  2. The external transaction coordinator would have to keep the state anyway and would know to not send any messages to prepared transactions.
  3. The external transaction coordinator would have to keep the state anyway and can pass this info to InitProducerId (via the keepPreparedTxn flag) instead of keeping this info in 2 places.
  4. The external transaction coordinator has already the knowledge of prepared transactions.

So we decided to keep the “implicit prepare” the same way we have it in Kafka today and avoid extra synchronous operations that would just duplicate the state that is kept in the external transaction coordinator.

HeartBeat RPC

Potentially, it could be good to distinguish between abandoned transactions (i.e. a producer started a 2PC transaction and then died) and just long running transactions.  We could add a heartbeat RPC between the 2 cases.

HeartBeat RPC definitely sounds like a “good thing to do”.  It is not clear, though, what would be the cases when we need to handle these situations differently – a long running transaction is a concern regardless of whether it got abandoned or the application is still working on it (also the application might have a bug or etc.), the operator would still need to investigate what’s going on with the application and determine if it’s safe to abort the transaction without violating “prepared” guarantees.

So it doesn’t seem to justify extra complexity, and we just add tooling for inspection of any long running transactions.

Using TransactionTimeoutMs=MAX_INT Instead of Enable2Pc

Having a Boolean flag and then a timeout value seems redundant, we really need just either one of the other, so technically instead of adding an independent flag we could use a special timeout value to indicate that it’s a 2PC transaction.  This, however, would couple intent with specific implementation; an explicit Boolean seems to reflect the intent better.

Allowing keepPreparedTxn=true without 2PC

Without 2PC the notion of “prepared” transaction is subverted by the transaction.max.timeout.ms so Kafka cannot promise to keep transaction in-doubt until a decision is reached.  So from a purity perspective using keepPreparedTxn=true doesn’t reflect the semantics.

In practice, however, Flink already effectively has a way to keep a “prepared” transaction by using reflection, so if we want to support all the following properties:

  1. Get rid of reflection use in Flink and move to public API in new versions of Flink
  2. Support new versions of Flink in Kafka clusters that don’t want to grant 2PC privileges.

Then we need to enable using keepPreparedTxn=true even if 2PC is disabled.

  • No labels