Versions Compared

Key

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

Table of Contents

Status

Current state: Drafting Discussion

Discussion thread: TBD

JIRA: TBD

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

Motivation

Kafka Streams supports an interesting and innovative API for "peeking" into the internal state of running stateful stream processors from outside of the application, called Interactive Query (IQ). This functionality has proven invaluable to users over the years for everything from debugging running applications to serving low latency queries straight from the Streams runtime.

However, the actual interfaces for IQ were designed in the very early days of Kafka Streams, before the project had gained significant adoption, and in the absence of much precedent for this kind of API in peer projects. With the benefit of hindsight, we can observe several problems with the original design that we hope to address in a revised framework that will serve Streams users well for many years to come.

Problems

  1. IQ serves queries by presenting callers with a composite store interface, which encapsulates the fact that stores will in general be partitioned, and that a given instance may only host a subset of those partitions (if any).
    1. The cost of constructing this store interface is non-trivial, though many real-world use cases will only use the store to run one query and then discard it.
    2. The creation of the store is subject to any number of error conditions, so callers need to handle exceptions on calling KafkaStreams.store() .
    3. Once you have an IQ store reference, it is still subject to any number of transient and permanent conditions (such as rebalances having moved partitions off the local instance, Streams changing state, etc.), so callers also need to handle exceptions while running queries on their store, and be prepared to rebuild the store if it becomes invalid.
    4. Users desiring to query custom state stores need to produce a fairly detailed implementation of QueryableStoreType that details how to compose results from multiple partitions into one.
      1. In particular, if you want to plug a store with special query capabilities in to the Streams DSL (for example as the materialization of a KTable), the store must extend the appropriate store interface, for example KeyValueStore<Bytes,byte[]> . However, when it comes to exposing it in IQ, you will have to implement QueryableStoreType<YourStore> , and it requires you to produce an instance of YourStore that delegates to a list of ReadOnlyKeyValueStore<Bytes,byte[]>  returned by storeProvider.stores . However, the catch is that those delegate stores are not  instances of your store type! They will be instances of the internal class MeteredTimestampedKeyValueStore , which wraps other internal store types, which after a few more layers of wrapping contain YourStore at the bottom. Your only choice will be to create a completely separate implementation  of the YourStore interface that delegates all the ReadOnlyKeyValueStore methods to those wrapper layers, and then for the methods that are special to YourStore, you'll have to repeatedly cast and type-check the wrappers until you manage to get all the way down to the actual YourStore that can serve the query.
      2. In other words, it's so complicated that it might as well be impossible. Which makes it not surprising that no one has actually tried. I suspect that if we made it easier to extend the storage framework, we would see a bunch of new use cases pop up building on IQ in the future.
  2. IQ composes all locally present partitions into a unified response. For example, for queries that return an iterator, it builds a composite iterator that collates all locally available partitions' iterators into one.
    1. While this is useful for trivial use cases, it destroys useful information about the response:
      1. Callers don't know which partitions were included in the response.
      2. After iterating for some time, callers can't tell when individual partitions' iterations are complete. This is important if we experience a failure: partitions that are already complete don't need to repeat the query.
    2. In practice, partitions' responses can succeed or fail independently, but the composite response couples all responses to any individual partition's failure.
  3. Because IQ responses are simply the result type of whatever store method you invoke, it is not possible to attach extra metadata that is useful in the IQ context, specifically.
    1. Eg. We might want to add detailed trace information about which store layers or segments participated in the query, including execution time, cache hit/miss, etc. This kind of feature would be particularly useful when debugging performance, or when IQ is backing a service that uses distributed tracing, etc.
    2. Eg. We might want to add information about the precise state of the view we served: what was the input offset we last wrote into the store? What is the "current stream time" of the view we served? What was the state of the StreamTask when we served the query? Etc.
    3. These are just examples from various conversations about potentially useful IQ response metadata. The point is to illustrate the fact that we are missing opportunities by restricting the IQ response to be the simple value returned by the store methods that serve the query.
  4. Supporting new types of queries to the "standard" store interfaces is onerous, since it requires adding new methods to the store interfaces, which need to be overridden in dozens of utility implementations throughout the Streams codebase.
    1. Example: KIP-617: Allow Kafka Streams State Stores to be iterated backwards . This change involved four PRs (https://github.com/apache/kafka/pull/9137, https://github.com/apache/kafka/pull/9138, https://github.com/apache/kafka/pull/9139/files, https://github.com/apache/kafka/pull/9239), totaling 108 files and 6,000+ lines of code changed.
    2. Another example: KIP-614: Add Prefix Scan support for State Stores (which only edits the KeyValueStore). This change took two PRs (https://github.com/apache/kafka/pull/9508 and https://github.com/apache/kafka/pull/10052), totaling 19 files and 600+ lines of code changed.
  5. IQ forces all queries to compose the contents of the Record Cache (the write buffer for Processors) with the underlying bytes stores.
    1. Despite its name, the Record Cache is a write buffer, not a traditional read cache. Perhaps not surprisingly, its performance is not very good for arbitrary queries, since its primary purpose is to ensure that Processors always read their own writes while delaying operations like sending writes to the changelog and the underlying stores.
    2. We could invest in optimizing the Record Cache for queries, but we would probably find that the better approach is to separate the read and write workloads.
    3. Regardless of potential future optimizations in the Record Cache, merging the buffered writes with the underlying stores (as in the case of a table scan) will always require extra work, and it would be good to have an option to skip the record cache for IQ users.
    4. In contrast to Processors, IQ interactions can never write, so do not need any concept of "read your writes".

Unintended Consequences

In a nutshell, all those issues result in a system that isn't ideal from anyone's perspective:

...

In conclusion, and to clarify: IQ itself has been extremely successful and valuable to many people. I only harp on the above points to demonstrate a number of flaws that I think we can improve on to make it even more valuable and successful in the future.

Goals

To address the above pain points while preserving the positive aspects of IQ, I'd propose the following goals for this KIP:

  1. We should continue to offer a mechanism to peek into the internal state of Kafka Streams's stateful operations.
  2. We should recognize that querying state via IQ is a different use case from using a store in a Processor, and a different interface should therefore be on the table.
  3. Simple use cases should continue to be easy.
    1. Particularly, it should continue to be easy to just "query the store" and not worry about individual partitions, etc.
  4. More complex use cases should be possible and not too hard.
    1. Particularly, it should be possible to pick and choose partitions to query, and to handle independent partitions' responses independently.
    2. It should also be possible to define new store types and queries with a manageable level of complexity.
    3. It should be possible to tune queries for maximum performance.
  5. Contributing to and maintaining the code base should be relatively straightforward.

Proposed Changes

This KIP proposes a new framework for IQ, which we will refer to as "IQv2". It is outside the scope of this KIP to propose new mechanisms for every query that IQ supports today, as part of the purpose of this design is to be easily extensible (and we want to keep the KIP focused on the framework). However, we do propose to add a few specific queries, to flesh out the proposal and to give some purpose to the initial batch of implementation work.

...

  • This addresses Problem 2 because responses aren't required to be composable
  • It also creates room for partitions to report successful or failure responses independently, which addresses Unintended Consequence 1b
  • It also addresses Unintended Consequence 1c because we don't do extra work to combine partions' results unless we need to.

Special interface stability protocol

All of the new methods and classes will be marked as @Evolving  ("Compatibility may be broken at minor release") to allow for more rapid interaction as we flesh out the first generation of queries. For such changes, we will simply notify the KIP discussion thread and update this document. Our goal is to stabilize the interface as quickly as possible, ideally in the next major release.

Public Interfaces

This KIP will add several new methods, classes, and interfaces.

KafkaStreams modifications

IQv2 will continue to present its primary API via the KafkaStreams interface. The query  method itself is this API.

...

Code Block
languagejava
titleKafkaStreams
public class KafkaStreams implements AutoCloseable {
  ...

  /**
    * Run an interactive query against a state store.
    * <p>
    * This method allows callers outside of the Streams runtime to
    * access the internal state of stateful processors. See
    * https://kafka.apache.org/documentation/streams/developer-guide/interactive-queries.html
    * for more information.
    * /
  @Evolving
  public <R> InteractiveQueryResult<R> query(InteractiveQueryRequest<R> request);

  /**
    * Get a reference to the serdes used internally in a state store
    * for use with interactive queries. While many queries already
    * accept Java-typed keys and also return typed results, some
    * queries may need to handle the raw binary data stored in StateStores,
    * in which case, this method can provide the serdes needed to interpret
    * that data.
    * /
  @Evolving
  public <K, V> InteractiveQuerySerdes<K, V> serdesForStore(String storeName);

  ...
}

InteractiveQueryRequest

This is the main request object for IQv2. It contains all of the information required to execute the query. Note that, although this class is similar to the IQv1 request object StoreQueryParameters , we are proposing a new class to avoid unnecessary coupling between the old and new APIs' design goals.

...

Code Block
/**
  * The request object for Interactive Queries.
  * This is an immutable builder class for passing all required and
  * optional arguments for querying a state store in Kafka Streams.
  * <p>
  * @param <R> The type of the query result.
  */
@Evolving
public class InteractiveQueryRequest<R> {

  /**
    * First required argument to specify the name of the store to query
    */
  public static InStore inStore(final String name);

  public static class InStore {

    /**
      * Second required argument to provide the query to execute.
      */
    public <R> InteractiveQueryRequest<R> withQuery(final Query<R> query);
  } 

  /**
    * Optionally bound the current position of the state store
    * with respect to the input topics that feed it. In conjunction
    * with {@link InteractiveQueryResult#getPosition}, this can be
    * used to achieve a good balance between consistency and
    * availability in which repeated queries are guaranteed to
    * advance in time while allowing reads to be served from any
    * replica that is caught up to that caller's prior observations.
    * <p>
    * Note that the set of offsets provided in the bound does not determine
    * the partitions to query. For that, see {@link withPartitionsToQuery}.
    * Unrelated offsets will be ignored, and missing offsets will be treated
    * as indicating "no bound". 
    */
  public InteractiveQueryRequest<R> withPositionBound(PositionBound positionBound);

  /**
    * Optionally specify the partitions to include in the query.
    * If omitted, the default is to query all locally available partitions
    */
  public InteractiveQueryRequest<R> withPartitions(Set<Integer> partitions);

  /**
    * Query all locally available partitions
    */
  public InteractiveQueryRequest<R> withAllPartitions(); 


  /**
    * Instruct Streams to collect detailed information during query
    * execution, for example, which stores handled the query, how
    * long they took, etc.
    */
  public InteractiveQueryRequest<R> enableExecutionInfo();

  // Getters are also proposed to retrieve the request parameters

  public String getStoreName();
  public Query<R> getQuery();
  public PositionBound getPositionBound();
  public boolean executionInfoEnabled() 

  /**
    * Optional.of(empty set) indicates that no partitions will be fetched
    * Optional.of(non-empty set) indicate the specific partitions that will be fetched (if locally available)
    */
  public Set<Integer> getPartitions();

  /**
    * indicates that all locally available partitions will be fetched
    */
  public boolean isAllPartitions();

}

InteractiveQueryResult

This is the main response object for IQv2. It wraps the individual results, as well as providing a vehicle to deliver metadata relating to the result as a whole.

Code Block
/**
  * The response object for interactive queries.
  * It wraps the individual results, as well as providing a
  * vehicle to deliver metadata relating to the result as a whole.
  * <p>
  * @param <R> The type of the query result. 
  */
@Evolving 
public class InteractiveQueryResult<R> {
  /**
    * Constructor. Used by Kafka Streams, and may be useful for
    * tests as well.
    */
  public InteractiveQueryResult(Map<Integer /*partition*/, QueryResult<R>> partitionResults);

  /**
    * The query's result for each partition that executed the query.
    */
  public Map<Integer /*partition*/, QueryResult<R>> getPartitionResults();

  /**
    * Asserts that only one partition returns a result and extract the result.
    * Useful with queries that expect a single result.
    */
  public QueryResult<R> getOnlyPartitionResult()


  /**
    * The position of the state store at the moment it executed the
    * query. In conjunction
    * with {@link InteractiveQueryRequest#withPartitionBound}, this can be
    * used to achieve a good balance between consistency and
    * availability in which repeated queries are guaranteed to
    * advance in time while allowing reads to be served from any
    * replica that is caught up to that caller's prior observations.
    */ 
  public Position getPosition();
}

StateStore modifications

This is the essence of the proposal. Rather than modifying each store interface to allow new kinds of queries, we introduce a generic capability of stores to execute query objects. This allows stores to choose whether they accept or reject queries of a given type, the introduction of new queries, etc.

...

Code Block
public interface StateStore {
  ...

  /**
    * Execute a query. Returns a QueryResult containing either result data or
    * a failure.
    * <p>
    * If the store doesn't know how to handle the given query, the result
    * will be a {@link FailureReason#UNKNOWN_QUERY_TYPE}.
    * If the store couldn't satisfy the given position bound, the result
    * will be a {@link FailureReason#NOT_UP_TO_BOUND}.
    * @param query The query to execute
    * @param offsetBound The position the store must be at or past
    * @param collectExecutionInfo Whether the store should collect detailed execution info for the query
    * @param <R> The result type
    */
  @Evolving
  default <R> QueryResult<R> query(Query<R> query,
                                   PositionBound positionBound,
                                   boolean collectExecutionInfo) {
    // If a store doesn't implement a query handler, then all queries are unknown.
    return QueryResult.forUnknownQueryType(query, this);
  }
 
  ...
}

Query

This is the interface that all queries must implement. Part of what I'm going for here is to place as few restrictions as possible on what a "query" or its "result" is, so the only thing in this interface is a generic type indicating the result type, which lets Streams return a fully typed response without predetermining the response type itself.

...

Using this interface, store implementers can create any query they can think of and return any type they like, from a single value to an iterator, even a future. While writing the POC, I implemented three queries for KeyValue stores, which I'll include here as examples:

Proposed Query: KeyQuery

This query implements the functionality of the current KeyValueStore#get(key)  method:

...

We propose to add KeyQuery as part of the implementation of this KIP, in order to have at least one query available to flesh out tests, etc., for the framework. Other queries are deferred to later KIPs.

Proposed Query: RawKeyQuery

We also propose to add a "raw" version of the KeyQuery, which simply takes the key as a byte array and returns the value as a byte array. This will be used to implement the KeyQuery, since in Streams the Metered store layers would "terminate" the typed KeyQuery by invoking the serializer on the key and then to execute a RawKeyQuery on the lower layers with the serialized key. Those lower layers would return a serialized value, which the Metered store would deserialize and (using QueryResult#swapValue ) convert it to a typed vale to return for the KeyQuery's result.

...

Code Block
@Evolving
public class RawKeyQuery implements Query<byte[]> {
  public static RawKeyQuery withKey(final Bytes key);
  public static RawKeyQuery withKey(final byte[] key);
  public Bytes getKey();
}


Example query: RawScanQuery

This example demonstrates two variations on the first example:

...

Code Block
public class RawScanQuery implements Query<KeyValueIterator<Bytes, byte[]>> {
    private RawScanQuery() { }

    public static RawScanQuery scan() {
        return new RawScanQuery();
    }
}

// example usage

// since we're handling raw data, we're going to need the serdes
InteractiveQuerySerdes<Integer, ValueAndTimestamp<Integer>> serdes =
  kafkaStreams.serdesForStore("mystore");

// run the "scan" query
InteractiveQueryResult<KeyValueIterator<Bytes, byte[]>> scanResult =
  kafkaStreams.query(inStore("mystore").withQuery(RawScanQuery.scan()));

// This time, we'll get results from all locally available partitions.
Map<Integer, QueryResult<KeyValueIterator<Bytes, byte[]>>> partitionResults =
  scanResult.getPartitionResults();


// for this example, we'll just collate all the partitions' iterators
// together and print their data

List<KeyValueIterator<Bytes, byte[]>> iterators =
  partitionResults
    .values()
    .stream()
    .map(QueryResult::getResult)
    .collect(Collectors.toList());

// Using an example convenience method we could add to collate iterators' data
try (CloseableIterator<KeyValue<Bytes, byte[]>> collated = Iterators.collate(collect)) {
  while(collate.hasNext()) {
    KeyValue<Bytes, byte[]> next = collated.next();
    System.out.println(
      "|||" +
        " " + serdes.keyFrom(next.key.get()) +
        " " + serdes.valueFrom(next.value)
    );
  }
}

QueryResult

This is a container for a single partition's query result.

Code Block
@Evolving
public class QueryResult<R> {
  // wraps a successful result
  public static <R> QueryResult<R> forResult(R result);

  // returns a failed query result because the store didn't know how to handle the query.
  public static <R> QueryResult<R> forUnknownQueryType(Query<R> query, StateStore store);

  // returns a failed query result because the partition wasn't caught up to the desired bound.
  public static <R> QueryResult<R> notUpToBound(Position currentPosition, Position bound);

  // returns a failed query result because caller requested a "latest" bound, but the task was
  // not active and running.
  public static <R> QueryResult<R> notActive(String currentState);

  // Used by state stores that need to delegate to another store to run a query and then
  // translate the results. Does not change the execution info or any other metadata.
  public <NewR> QueryResult<NewR> swapResult(NewR newTypedResult);

  // If requested, stores should record
  // helpful information, such as their own class, how they executed the query,
  // and the time they took.
  public void addExecutionInfo(String executionInfo);

...
}

Position

A class representing a processing state position in terms of its inputs: a vector or (topic, partition, offset) components.

Code Block
@Evolving
public interface Position {

  // Create a new Position from a map of topic -> partition -> offset
  static Position fromMap(Map<String, Map<Integer, Long>> map);

  // Create a new, empty Position
  static Position emptyPosition();

  // Return a new position based on the current one, with the given component added
  Position withComponent(String topic, int partition, long offset);

  // Merge all the components of this position with all the components of the other
  // position and return the result in a new Position
  Position merge(Position other);

  // Get the set of topics included in this Position
  Set<String> getTopics();

  // Given a topic, get the partition -> offset pairs included in this Position
  Map<Integer, Long> getBound(String topic);
}

PositionBound

A class bounding the processing state Position during queries. This can be used to specify that a query should fail if the locally available partition isn't caught up to the specified bound. "Unbounded" places no restrictions on the current location of the partition, and "latest" indicates that only a running active replica should reply to the query.

Code Block
@Evolving
public interface PositionBound {

  // Create a new Position bound representing only the latest state
  static PositionBound latest();

  // Create a new Position bound representing "no bound"
  static PositionBound unbounded();

  // Create a new, empty Position
  static PositionBound at(Position position);

  // Check whether this is a "latest" Position bound
  boolean isLatest();

  // Check whether this is an "unbounded" Position bound
  boolean isUnbounded();

  // Get the Position (if it's not unbounded or latest)
  Position position();
}


Compatibility, Deprecation, and Migration Plan

Since this is a completely new set of APIs, no backward compatibility concerns are anticipated. Existing IQ usages will be free to migrate to the new IQv2 queries as they become available. Deprecation of the existing IQ API will be considered and proposed in a later KIP after IQv2 is determined to be feature complete, ergonomic, and performant enough to serve existing use cases.

...

Since nothing is deprecated in this KIP, users have no need to migrate unless they want to.

Rejected Alternatives

Incrementally improve the current IQ API

Some of the motivating problems could be addressed within the scope of the current IQ API, but many are inherent in the current API's design. The full set of problems convinced us that it's time for a new API.

Defer the Position aspects to a later KIP

This would certainly simplify this KIP, but it would also mean we need to update the StateStore interface twice. If the Position proposal generates too much controversy, we will split it into a separate KIP.

Defer the Position functionality to individual StateStore/Query implementations

The IQv2 design leaves the door open for Queries and StateStores to coordinate to provide all kinds of functionality on top of "just" returning responses to queries. For example, new Queries and StateStores can be implemented to return asynchronous Future<R>  results. So perhaps it is also a good idea to push the idea of bounding the Position into the "user space" of Query/StateStore implementations as well. We think that it could be done, but there are also advantages to making it a framework feature. The biggest one is simply uniformity. The framework Position tracking and bounding can ensure a consistent API across all query and store implementations, whereas pushing it to the implementations would make it merely conventional, raising future interoperability concerns. Position data also needs to be transmitted to standby replicas and made fault-tolerant, all of which becomes more complex in "user space" than in the framework.

Improve Query dispatch performance.

This was inspired by the fact that we don't have a great mechanism to dispatch Queries to their execution methods inside the state store. Presently, we have to either have a sequence of if/else type checks or a switch statement in the StateStore#execute  implementation, or some other approach like a HashMap of Query → method. Java 17 has the ability to switch over the class itself (see https://openjdk.java.net/jeps/406), but that's obviously a long way off for Apache Kafka.

Host the execution logic in the Query instead of the StateStore

Since the number of Query implementations will probably be much more than the number of StateStore implementations, we can move the type-checks from StateStore to Query by replacing StateStore#execute(Query)  with Query#executeOn(StateStore) . Then, instead of having StateStore implementations define how to behave for each query, we would have the Queries define how to run on each StateStore.

...

There's a more sophisticated version of this that allows both the Query and the StateStore to execute the query, so that the Query can try its (hopefully) more efficient dispatch first, and then defer to the StateStore's more flexible dispatch. This might be the best of both worlds in terms of dispatch performance, but it's quite complicated. We will consider this we are unable to make the current proposal perform acceptably.

Implement the Visitor pattern instead of checking the type of the Query

Although this proposal has some things in common with a Visitor-pattern situation, there are two different things that spoil this as a solution.

...