You are viewing an old version of this page. View the current version.
Compare with Current
View Page History
Version 1
Next »
This page is meant as a template for writing a KIP. To create a KIP choose Tools->Copy on this page and modify with your content and replace the heading with the next KIP number and a description of your issue. Replace anything in italics with your own description.
Status
Current state: Under Discussion
Discussion thread:
JIRA: KAFKA-15257
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
The main goal is supporting interactive queries in presence of versioned state stores (KIP-889) in AK. This KIP is the successor of KIP-960 and discusses single-key, multi-timestamp queries. Other types of IQs are explained in the following KIP (KIP-968)
Key Queries with multiple timestamps:
- single-key query with upper bound timestamp
- single-key query with lower bound timestamp
- single-key query with timestamp range
- single-key all versions query
Public Interfaces
In this KIP we propose a public classes, VersionedKeyQuery. Moreover, the public interface ValueIterator is used to iterate over different values that are returned from a single-key query (each value corresponds to a timestamp).
Proposed Changes
For single-key queries, VersionedKeyQuery and ValueIterator classes will be used.
package org.apache.kafka.streams.state;
/**
* Iterator interface of {@link Value}.
* <p>
* Users must call its {@code close} method explicitly upon completeness to release resources,
* or use try-with-resources statement (available since JDK7) for this {@link Closeable} class.
* Note that {@code remove()} is not supported.
*
* @param <V> Type of values
*/
public interface ValueIterator<V> extends Iterator<V>, Closeable {
@Override
void close();
/**
* Peek the next value without advancing the iterator
* @return the value that would be returned from the next call to next
*/
V peek();
}
VersionedKeyQuery class
- The methods are composable. Therefore, the meaningless combinations such as withKey(k1).asOf(t1).allVersions() end up throwing a RunTimeException (for example NotSupportedException).
- Defining a query with time range (empty, t1] will be translated into [0, t1]
- Defining a query with time range (t1, empty) will be translated into [t1, MAX)
package org.apache.kafka.streams.query;
/**
* Interactive query for retrieving a set of records with the same specified key and different timestamps within the specified time range.
*/
@Evolving
public final class VersionedKeyQuery<K, V> implements Query<ValueIterator<VersionedRecord<V>>> {
private final K key;
private final Optional<Instant> fromTimestamp;
private final Optional<Instant> asOfTimestamp;
private final boolean isAscending;
private VersionedKeyQuery(
final K key,
Optional<Instant> fromTimestamp,
Optional<Instant> asOfTimestamp,
boolean isAscending) {
this.key = Objects.requireNonNull(key);
this.fromTimestamp = fromTimestamp;
this.asOfTimestamp = asOfTimestamp;
this.isAscending = isAscending;
}
/**
* Creates a query that will retrieve the set of records identified by {@code key} if any exists
* (or {@code null} otherwise).
* @param key The key to retrieve
* @param <K> The type of the key
* @param <V> The type of the value that will be retrieved
*/
public static <K, V> VersionedKeyQuery<K, V> withKey(final K key);
/**
* Specifies the starting time point for the key query.
* The key query returns all the records that are valid in the time range starting from the timestamp {@code fromTimestamp}.
* @param fromTimestamp The starting time point
*/
public VersionedKeyQuery<K, V> from(Instant fromTimestamp);
/**
* Specifies the ending time point for the key query.
* The key query returns all the records that have timestamp <= {@code asOfTimestamp}.
* @param asOfTimestamp The ending time point
*/
public VersionedKeyQuery<K, V> asOf(Instant asOfTimestamp);
/**
* Specifies the starting and ending points of the key query as MIN and MAX respectively.
* Therefore, the query returns all the existing records in the state store with the specified key.
* @throws RuntimeException if {@code fromTimestamp} or {@code asOfTimestamp} have been already
* specified.
*/
public VersionedKeyQuery<K, V> allVersions();
/**
* Specifies the order of the returned records by the query as descending by timestamp.
*/
public VersionedKeyQuery<K, V> descending();
/**
* The key that was specified for this query.
*/
public K getKey();
/**
* The starting time point of the query, if specified
*/
public Optional<Instant> getFromTimestamp();
/**
* The ending time point of the query, if specified
*/
public Optional<Instant> getAsOfTimestamp();
/**
* @return true if the query returns records in ascending order of timestamps
*/
public boolean isAscending ();
}
Examples
The following example illustrates the use of the VersionedKeyQuery class to query a versioned state store.
final VersionedKeyQuery<Integer, Integer> query = VersionedKeyQuery.withKey(1).allVersions();
final StateQueryRequest<ValueIterator<VersionedRecord<Integer>>> request =
inStore("my_store").withQuery(query);
final StateQueryResult<ValueIterator<VersionedRecord<Integer>>> versionedKeyResult = kafkaStreams.query(request);
// Get the results from all partitions.
final Map<Integer, QueryResult<ValueIterator<VersionedRecord<Integer>>>> partitionResults = versionedKeyResult.getPartitionResults();
for (final Entry<Integer, QueryResult<ValueIterator<VersionedRecord<Integer>>>> entry : partitionResults.entrySet()) {
try (final ValueIterator<VersionedRecord<Integer>> iterator = entry.getValue().getResult()) {
while (iterator.hasNext()) {
final VersionedRecord<Integer> record = iterator.next();
Long timestamp = record.timestamp();
Integer value = record.value();
}
}
}
Compatibility, Deprecation, and Migration Plan
- Since this is a completely new set of APIs, no backward compatibility concerns are anticipated.
- Since nothing is deprecated in this KIP, users have no need to migrate unless they want to.