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

Compare with Current View Page History

« Previous Version 3 Next »

Status

Current state: Under Discussion

Discussion thread: here

JIRA: here

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

Motivation

Right now, EOS has to delete all data from the local state stores on crash failure because the state stores might be in a partially updated state after crash failure. The partial update of a state store can happen during a crash failure because the changes to the local state are not atomic with respect to Kafka Streams commit. If an application with EOS crashes between commits, it cannot reset the state to the previously committed, so it wipes the state stores and replays the changelog from scratch.

This KIP proposes making writes to the state stores transactional, so that they atomically commit only after the corresponding changes are committed to the changelog topic. As a result, Streams applications configured with EOS will no longer need to wipe the state stores on crash failure.

Public Interfaces

Changed:

  • org.apache.kafka.streams.processor.StateStore
  • org.apache.kafka.streams.StreamsConfig
  • org.apache.kafka.streams.StoreQueryParameters

Proposed Changes

Overview

This section introduces the overview of the proposed changes. The following sections will cover the changes in behavior, configuration, and interfaces in detail.

This KIP proposes introducing an enabled by default feature flag (statestore.transactional.enabled)  that makes writes to state stores transactional, meaning they apply atomically after the applications commits to the changelog topic. The transactional guarantees come with non-zero overhead, so the feature flag acts as a safety switch if Kafka Streams users decide to opt-out. This feature presents a trade-off between increased read, write, and memory amplifications and time to recover from crash failure in the case of EOS.

When the feature flag is on, and the state store supports transactions (checked via new boolean method StateStore#transactional), it applies changes atomically only after they were committed to the changelog topic. Transactional state stores do not delete the checkpoint file and the underlying data in the case of EOS. Instead, they roll forward from the changelog topic on recovery. When the feature flag is off or the store does not support transactions, EOS recovery behaves as it does now - the task will delete the checkpoint file and recover from the changelog.

This KIP does not introduce separate methods to begin and commit transactions. Instead, a transactional state store commits a currently open transaction on StateStore#flush if the feature flag is true. The reasons for that behavior are the following:

  1. There is always a single writer in Kafka Streams workloads, and all writes must go to a single currently open transaction. 
  2. There is always a single reader that queries dirty state from a single open transaction.
  3. The state stores already explicitly call flush before writing to the checkpoint file to make uncommitted changes durable. Adding separate methods will create room for error, such as a missing call to beginTxn or commitTxn, or flushing before committing.


There are multiple ways to implement state store transactions that present different trade-offs. This KIP includes a single implementation via a built-in RocksDB indexed batches and adds a configuration option (statestore.transactional.mechanism) to add other implementations in the future in a backward-compatible way.

Interface Changes

StateStore

StateStore.java
/**
* Return if the storage supports transactions or not.
*
* @return {@code true} if the storage supports transactions, {@code false} otherwise
*/
default boolean transactional() {
   return false;
}


/**
* Flush any cached data. When {@link #transactional()} returns true, flush commits dirty writes.
*/
void flush();

The flush signature did not change, but the way it is expected to work when transactional() is true does, so I am including it here as well:

Configuration changes

StreamsConfig:

  • statestore.transactional.enabled (true by default) - enables transactional behavior for the state stores. When false, the state stores behave as they do now.
  • statestore.transactional.mechanism (rockdb_indexbatch by default) - specifies the means to implement transactional writes. rockdb_indexbatch is the only valid configuration value for now.

StoreQueryParameters:

  • readCommitted (false by default) - controls whether interactive queries read uncommitted or committed data. 

Behavior changes

If statestore.transactional.enabled is true and the underlying state store implements transactions (i.e., StateStore#transactional() returns true), then the store writes via the transactional mechanism specified by statestore.transactional.mechanism. Reads via ReadOnlyKeyValueStore methods return uncommitted data from the ongoing transaction.

If statestore.transactional.enabled is true, but the store does not support transactions, the corresponding stream task falls back to the non-transactional behavior.

Interactive queries return either committed or uncommitted results, according to enable.iq.read_committed. 

A transactional state store opens the first transaction during initialization. It commits on StateStore#flush - first, the store commits the transaction, then flushes, then starts a new transaction. 

There are several places where StreamTask, ProcessorStateManager, and TaskManager check if EOS is enabled and then it deletes the checkpoint file on crash failure, specifically, when:

  • StreamTask resumes processing (link)
  • ProcessorStateManager initializes state stores from offsets from checkpoint (link1, link2)
  • StreamTask writes offsets to the checkpoint file on after committing (link)
  • TaskManager handles revocation (link)

The if-EOS condition should only apply in all these cases if the feature flag is off or state stores are not transactional.

RocksDB

When statestore.transactional.mechanism=rockdb_indexbatch, Kafka Streams will make the writes to the built-in RocksDB state store transactional by using  WriteBatchWithIndex, which is similar to WriteBatch already used segment stores, except it also allows reading uncommitted data.

The advantage of this approach is that it uses the RocksDB built-in mechanism to ensure transactionality and offers the smallest possible write amplification overhead.

The disadvantage of this approach is that all uncommitted writes must fit into memory. In practice, RocksDB developers recommend the batches to be no larger than 3-4 megabytes (link).

A considered alternative is OptimisticTransactionDB, which offers more guarantees than Kafka Streams needs, specifically - ensures that there were no write conflicts between concurrent transactions before committing. There are no concurrent transactions in Kafka Streams, so there is no reason to pay for the associated overhead.

Compatibility, Deprecation, and Migration Plan

Applications using Streams DSL and a built-in RocksDB state store implementation will get transactional state stores enabled by default. They can opt out by disabling the feature flag. 

Applications using Processor API will work as they do now. This will happen because the default implementation of StateStore#transactional returns false, so the relevant code in StreamTask, ProcessorStateManager, etc. will fall back to the non-transactional behavior.

Proposed changes are source compatible and binary incompatible with previous releases.

Test Plan

  1. Changes not committed to the changelog topic are discarded on crash failure.
  2. Changes committed to the changelog topic, but not committed to the state store are rolled forward.

Rejected Alternatives

Transactions via Secondary State Store for Uncommitted Changes

In this alternative, Kafka Streams opens two stores instead of one - a temporary store and a regular store. All uncommitted writes go to the temporary store. Once the task flushes, the temporary store creates a commit file, an empty file that indicates that the corresponding file is ready to commit, and writes stored records to the regular store. It truncates the store and deletes the commit file once it is finished. 

On crash failure, the task checks if the temporary store and the commit file exist. If they do, the task repeats the commit process described above. Otherwise, it truncates the temporary store.

The major advantage of this approach is that the temporary state store can optionally use the available disk space if the writes do not fit into the in-memory buffer. 

The disadvantages are:

  • It doubles the number of open state stores 
  • It potentially has higher write and read amplification due to uncontrolled flushes of the temporary state store.

For completeness, below are the details on the changes required to support this implementation.

Interface changes

StoreSupplier:

  • Add T get(final String suffix) that returns a state store whose name ends with a suffix. We need this method to create temporary state stores.

TransactionalKeyValueByteStore:

  • A constructor accepts a DB and a store supplier for the tmp store.
  • On init, the state store configures depending on the state of the feature flag. If it is set to false, then TransactionalKeyValueByteStore just forwards all methods to inner. Otherwise, it creates a temporary store and implements behavior changes described below.

Behavior changes

All writes and deletes go to the temporary store. Reads query the temporary store and if the data is missing, query the regular store. Range reads query both stores and return a KeyValueIterator that merges the results.

Similar to CachingKeyValueStore and ChangeLoggingKeyValueBytesStore, classes extending AbstractStoreBuilder will wrap created state stores with TransactionalKeyValueByteStore.

Method to control transaction lifecycle in StateStore

A considered alternative is to introduce methods like StateStore#beginTxn and StateStore#commitTxn to manage transactions lifecycle. I don’t think they are necessary due to stream workloads specifics - there is always a single transaction for a given task and that transaction commits only after the commit to the changelog. Moreover, explicit method calls to begin and commit a transaction introduce possible invalid states, like skipping beginTxn before committing, beginning a transaction multiple times, committing after flushing, etc.


  • No labels