Versions Compared

Key

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


(Working in Progress)

 

 

Table of Contents

Status

Current state:  Under Discussion Accepted(vote)

Discussion thread: here

JIRA

Jira
serverASF JIRA
columnskey,summary,type,created,updated,due,assignee,reporter,priority,status,resolution
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-5876

...

Motivation

Currently, IQ throws InvalidStateStoreException for any types of error, that means a user cannot handle different types of error.

Because of that, we should throw different exceptions for each type. 

Proposed Changes

There append three new exceptions:To distinguish different types of error, we need to handle all InvalidStateStoreException better during these public methods invoked. The main change is to introduce new exceptions that extend from InvalidStateStoreException. InvalidStateStoreException is not thrown at all anymore, but only new sub-classes.

Code Block
languagejava
public class StateStoreClosedExceptionStreamsNotStartedException extends InvalidStateStoreException
public class StreamsRebalancingException extends InvalidStateStoreException
public class StateStoreMigratedException extends InvalidStateStoreException
public class StateStoreRetryableExceptionStateStoreNotAvailableException extends InvalidStateStoreException
public class UnknownStateStoreException extends InvalidStateStoreException
public class StateStoreFailExceptionInvalidStateStorePartitionException extends InvalidStateStoreException

Three categories exception throw to user


  • StreamsNotStartedException: will be thrown when stream thread state is CREATED, the user can retry until to RUNNING.
  • StreamsRebalancingException: will be thrown when stream thread is not running and stream state is REBALANCINGStateStoreRetriableException: The application instance in the state of rebalancing, the user just need retry and wait until rebalance finished (RUNNING).
  • StateStoreMigratedException: The store got migrated and not hosted in application instance, the users will be thrown when state store already closed and stream state is RUNNING. The user need to rediscover the store .
  • StateStoreFailException: Fatal error when access state store, the user cannot retry or rediscover.

...

  • and cannot blindly retry as the store handle is invalid and a new store handle must be retrived.
  • StateStoreNotAvailableException: will be thrown when state store closed and stream state is PENDING_SHUTDOWN / NOT_RUNNING / ERROR. The user cannot retry when this exception is thrown.
  • UnknownStateStoreException: will be thrown when passing an unknown state store. The user cannot retry when this exception is thrown.
  • InvalidStateStorePartitionException: will be thrown when user requested partition is not available on the stream instance.


The following is the public method methods that users will call to get state store instance:

  • KafkaStreams
    • stores()
  • ReadOnlyKeyValueStore(CompositeReadOnlyKeyValueStore)
    • get(k)
    • range(from, to)
    • all()
    • approximateNumEntries()
  • ReadOnlySessionStore(CompositeReadOnlySessionStore)
    • fetch(k)
    • fetch(from, to)
  • ReadOnlyWindowStore(CompositeReadOnlyWindowStore)
    • fetch(k, rf, tt)
    • fetch(from, to, rf, tt)
    • all()
    • fetchAll()
  • KeyValueIterator(DelegatingPeekingKeyValueIterator)
    • next()
    • hasNext()
    • peekNextKey()

During user call one of above methods, we should check KafkaStreams state by the following rule when InvalidStateStoreException is thrown:

  • If state is RUNNING or REBALANCING
    • StateStoreClosedException: should be wrapped to StateStoreRetriableException
    • StateStoreMigratedException: should not be wrapped, directly thrown
  • if state is PENDING_SHUTDOW or ERROR or NOT_RUNNING
    • wrap InvalidStateStoreException to StateStoreFailException

 

Call Trace

Expand
titleCall trace 1: KafkaStreams#store()
  • KafkaStreams#store() (v)
    • QueryableStoreProvider#getStore() (v)
      • GlobalStateStoreProvider#stores() (v)
      • StreamThreadStateStroeProvider#stores() (v)
Expand
titleCall trace 2: ReadOnlyKeyValueStore#get()
  • CompositeReadOnlyKeyValueStore#get() (v)
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
    • MeteredKeyValueBytesStore#get()
      • InnerMeteredKeyValueStore#get()
        • CachingKeyValueStore#get()
          • AbstractStateStore#validateStoreOpen() (v)
            • RocksDBStore#isOpen()
        • CachingKeyValueStore#getInternal()
          • ChangeLoggingKeyValueBytesStore#get()
            • RocksDBStore#get()
              • RocksDBStore#validateStoreOpen() (v)
            • RocksDBStore#getInternal()
Expand
titleCall trace 3: ReadOnlyKeyValueStore#range()
  • CompositeReadOnlyKeyValueStore#range() (v)
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
    • return new DelegatingPeekingKeyValueIterator<>()
  • DelegatingPeekingKeyValueIterator#hasNext() (v)
    • CompositKeyValueIterator#hasNext()
      • NextIteratorFunction#apply() (v)
        • MeteredKeyValueBytesStore#range()
          • InnerMeteredKeyValueStore#range()
            • CachingKeyValueStore#range()
              • AbstractStateStore#validateStoreOpen() (v)
              • ChangeLoggingKeyValueBytesStore#range()
                • RocksDBStore#range()
                  • RocksDBStore#validateStoreOpen() (v)
                  • return new RocksDBRangeIterator()
              • return new MergedSortedCacheKeyValueBytesStoreIterator()
            • return new MeteredKeyValueIterator()
          • return
        • return
      • return
    • CompositKeyValueIterator#next()
      • MeteredKeyValueIterator#next()
        • MergedSortedCacheKeyValueBytesStoreIterator#next()
          AbstractMergedSortedCacheStoreIterator#next()
          • RocksDBRangeIterator#hasNext()
            RocksDBIterator#hasNext() (v)
              • RocksIterator#isValid()
          • AbstractMergedStortedCacheStoreIterator#nextSrtoreValue()
            • RocksDBRangeIterator#next()
              RocksDBIterator#next()
              • RocksDbIterator#hasNext() (v)
              • RocksDbIterator#getKeyValue()
              • RocksIterator#next()
              • return keyvalue entry
            • return
          • return
        • return outerkeyvalue
      • return
    • return
  • DelegatingPeekingKeyValueIterator#next()
    • return keyvalue
Expand
titleCall trace 4: ReadOnlyKeyValueStore#all()
  • CompositeReadOnlyKeyValueStore#all() (v)
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
    • return new DelegatingPeekingKeyValueIterator<>()
  • DelegatingPeekingKeyValueIterator#hasNext() (v)
    • CompositKeyValueIterator#hasNext()
      • NextIteratorFunction#apply() (v)
        • MeteredKeyValueBytesStore#all()
          • InnerMeteredKeyValueStore#all()
            CachingKeyValueStore#all()
            • AbstractStateStore#validateStoreOpen() (v)
              • ChangeLoggingKeyValueBytesStore#all()
                • RocksDBStore#all()
                  • RocksDBStore#validateStoreOpen() (v)
                  • return new RocksDBIterator()
                • return
              • return new MergedSortedCacheKeyValueBytesStoreIterator()
            • return new MeteredKeyValueIterator()
          • return
        • return
      • return
    • CompositKeyValueIterator#next()
      • MeteredKeyValueIterator#next()
        • MergedSortedCacheKeyValueBytesStoreIterator#next()
          AbstractMergedSortedCacheStoreIterator#next()
          • MemoryLRUCacheBytesIterator.hasNext()
          • DelegatingPeekingKeyValueIterator.hasNext() (v)
          • AbstractMergedSortedCacheStoreIterator#nextStoreValue()
            • DelegatingPeekingKeyValueIterator#next()
              • DelegatingPeekingKeyValueIterator#hasNext() (v)
            • return
          • return
        • return outerkeyvalue
      • return
    • return
  • DelegatingPeekingKeyValueIterator#next()
    • return keyvalue

 

Expand
titleCall trace 5: ReadOnlyKeyValueStore#approximateNumEntries()
  • CompositeReadOnlyKeyValueStore#approximateNumEntries() (v)
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
    • MeteredKeyValueBytesStore#approximateNumEntries()
      • InnerMeteredKeyValueStore#approximateNumEntries()
        • CachingKeyValueStore#approximateNumEntries()
          • AbstractStateStore#validateStoreOpen() (v)
            • RocksDBStore#isOpen()
          • RocksDBStore#approximateNumEntries()
            • RocksDBStore#validateStoreOpen() (v)
            • return value
          • return
        • return
      • return
    • return total

...

titleCall trace 6: ReadOnlySessionStore#fetch(key)

...

  • StreamThreadStateStoreProvider#stores() (v)

...

  • MeteredSessionStore#findSessions()
    • CachingSessionStore#findSessions()
      • AbstractStateStore#validateStoreOpen() (v)
      • ChangeLoggingSessionBytesStore#findSessions()
        • RocksDBSessionStore.findSessions(k)
          • RocksDBSessionStore.findSessions(from, to)
            • RocksDBSegmentedBytesStore#fetch()
              • SessionKeySchema#segmentsToSearch()
                • Segments#segments()
                  • Segments#getSegment()
                    • ConcurrentHashMap#get()
                    • Segments#isSegment()
                    • return segment
                  • retiurn segments
                • return
              • return new SegmentIterator()
            • return new WrappedSessionStoreIterator()
          • return
        • return
      • return new MergedSortedCacheSessionStoreIterator()
    • return new MeteredWindowedKeyValueIterator()
  • return

...

    • @Deprecated store(storeName, queryableStoreType)
    • store(storeQureyParams)
Info

All the above methods could be throw exceptions:

StreamsNotStartedException, StreamsRebalancingException, StateStoreMigratedException, StateStoreNotAvailableException, UnknownStateStoreException, InvalidStateStorePartitionException


The following is the public methods that users will call to get store values:

...

  • interface ReadOnlyKeyValueStore(class CompositeReadOnlyKeyValueStore)
    • get(key)
    • range(from, to)

...

    • all()

...

    • approximateNumEntries()

...

  • interface ReadOnlySessionStore(class CompositeReadOnlySessionStore)
    • fetch(key)
    • fetch(from, to)

...

  • interface ReadOnlyWindowStore(class CompositeReadOnlyWindowStore)

...

  • WrappedSessionStoreIterator#next()
    • SegmentIterator#next()
      • RocksDBRangeIterator#next()
        RocksDbIterator#next()
        • RocksDbIterator#getKeyValue()
        • RocksIterator#next()
        • return entry
      • return
    • return
  • return

...

titleCall trace 7: ReadOnlySessionStore#fetch(from, to)

...

  • WrappingStoreProvider#stores() (v)
    • StreamThreadStateStoreProvider#stores() (v)
  • return new DelegatingPeekingKeyValueIterator<>()

...

    • fetch(key, time)
    • fetch(key,

...

    • from, to)

...

    • fetch(from, to,

...

    • fromTime,

...

    • toTime)

...

  • CachingSessionStore#validStoreOpen()
    AbstractStateStore#validStoreOpen() (v)
    • ChangeLoggingSessionBytesStore#isOpen()
      • RocksDBSessionStore#isOpen()
        AbstractStateStore#isOpen()
        • RocksDBSegmentedBytesStore#isOpen()
        • return
      • return
    • return
  • ChangeLoggingSessionBytesStore#findSesisons()
    • RocksDBSessionStore#findSessions()
      • RocksDBSegmentedBytesStore#fetch()
        • SessionKeySchema#segmentsToSearch()
          • Segments#segments() (v)
            • RocksDBStore#isOpen()
          • return new SegmentIterator()
        • return new WrappedSessionStoreIterator()
      • return
    • return
  • return new MergedSortedCacheSessionStoreIterator()

...

  • MergedSortedCacheSessionStoreIterator#hasNext()
    AbstractMergedSortedCacheStoreIterator#hasNext()
    • FilteredCacheIterator#hasNext()
    • WrappedSessionStoreIterator#hasNext()
      • SegmentIterator#hasNext()
        • Segment.range(from, to)
          RocksDBStore.range(from, to)
    • all()
    • fetchAll(from, to)
    • @Deprecated fetch(key, timeFrom, timeTo)
    • @Deprecated fetch(from, to, timeFrom, timeTo)
    • @Deprecated fetchAll(timeFrom, timeTo)
  • interface KeyValueIterator(class DelegatingPeekingKeyValueIterator)
    • next()
    • hasNext()
    • peekNextKey()
Info

All the above methods could be throw following exceptions: 

StreamsRebalancingExceptionStateStoreMigratedException, StateStoreNotAvailableException, InvalidStateStorePartitionException


Compatibility, Deprecation, and Migration Plan

  • All new exceptions extend from InvalidStateStoreException, this change will be fully backward compatible.

Rejected Alternatives

None.


G
M
T
                • RocksDBStore.validateStoreOpen() (v)
                • return new RocksDBRangeIterator()
              • return
            • return
          • return
        • return
      • return
    • CompositeKeyValueIterator#next()
    • return
  • DelegatingPeekingKeyValueIterator#next()
    • return keyvalue

 

Changed Classes

Code Block
languagejava
// TODO: 
// During user call one of following methods, we should check KafkaStreams state by the following rule when InvalidStateStoreException is thrown:
// * If state is RUNNING or REBALANCING
//   * StateStoreClosedException: should be wrapped to StateStoreRetriableException
//   * StateStoreMigratedException: should not be wrapped, directly thrown
// * If state is PENDING_SHUTDOW or ERROR or NOT_RUNNINGwrap InvalidStateStoreException to StateStoreFailException
//
 
public class KafkaStreams {
    public <T> T store(final String storeName, final QueryableStoreType<T> queryableStoreType);
}
 
public class CompositeReadOnlyKeyValueStore<K, V> implements ReadOnlyKeyValueStore<K, V> {
    public V get(final K key);
    public KeyValueIterator<K, V> range(final K from, final K to);
    public KeyValueIterator<K, V> all();
    public long approximateNumEntries();
}
 
public class CompositeReadOnlySessionStore<K, V> implements ReadOnlySessionStore<K, V> {
    public KeyValueIterator<Windowed<K>, V> fetch(final K key);
    public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to);
}
 
public class CompositeReadOnlyWindowStore<K, V> implements ReadOnlyWindowStore<K, V> {
    public WindowStoreIterator<V> fetch(final K key, final long timeFrom, final long timeTo);
    public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo);
    public KeyValueIterator<Windowed<K>, V> all();
    public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo);
}
 
class DelegatingPeekingKeyValueIterator<K, V> implements KeyValueIterator<K, V>, PeekingKeyValueIterator<K, V> {
    public synchronized boolean hasNext();
    public synchronized KeyValue<K, V> next();
    public KeyValue<K, V> peekNext();
}
 
Code Block
languagejava
public interface QueryableStoreType<T> {
    // TODO: pass stream instance parameter
    T create(final KafkaStreams streams, final StateStoreProvider storeProvider, final String storeName);
}
Code Block
languagejava
titleQueryableStoreProvider#getStore()
collapsetrue
public <T> T getStore(final String storeName, final QueryableStoreType<T> queryableStoreType) {
    final List<T> globalStore = globalStoreProvider.stores(storeName, queryableStoreType);
    if (!globalStore.isEmpty()) {
        return queryableStoreType.create(new WrappingStoreProvider(Collections.<StateStoreProvider>singletonList(globalStoreProvider)), storeName);
    }
    final List<T> allStores = new ArrayList<>();
    for (StateStoreProvider storeProvider : storeProviders) {
        allStores.addAll(storeProvider.stores(storeName, queryableStoreType));
    }
    if (allStores.isEmpty()) {
        // TODO: Replace with StateStoreMigratedException
        throw new InvalidStateStoreException("the state store, " + storeName + ", may have migrated to another instance.");
    }
    return queryableStoreType.create(
            new WrappingStoreProvider(storeProviders),
            storeName);
}
Code Block
languagejava
titleGlobalStateStoreProvider#stores()
collapsetrue
public <T> List<T> stores(final String storeName, final QueryableStoreType<T> queryableStoreType) {
    final StateStore store = globalStateStores.get(storeName);
    if (store == null || !queryableStoreType.accepts(store)) {
        return Collections.emptyList();
    }
    if (!store.isOpen()) {
        // TODO: Replace with StateStoreClosedException
        throw new InvalidStateStoreException("the state store, " + storeName + ", is not open.");
    }
    return (List<T>) Collections.singletonList(store);
}
Code Block
languagejava
titleStreamThreadStateStoreProvider#stores()
collapsetrue
 public <T> List<T> stores(final String storeName, final QueryableStoreType<T> queryableStoreType) {
    if (streamThread.state() == StreamThread.State.DEAD) {
        return Collections.emptyList();
    }
    if (!streamThread.isRunningAndNotRebalancing()) {
        // TODO: Replace with StateStoreMigratedException
        throw new InvalidStateStoreException("Cannot get state store " + storeName + " because the stream thread is " +
                streamThread.state() + ", not RUNNING");
    }
    final List<T> stores = new ArrayList<>();
    for (Task streamTask : streamThread.tasks().values()) {
        final StateStore store = streamTask.getStore(storeName);
        if (store != null && queryableStoreType.accepts(store)) {
            if (!store.isOpen()) {
                // TODO: Replace with StateStoreClosedException
                throw new InvalidStateStoreException("Cannot get state store " + storeName + " for task " + streamTask +
                        " because the store is not open. The state store may have migrated to another instances.");
            }
            stores.add((T) store);
        }
    }
    return stores;
}
Code Block
languagejava
titleWrappingStoreProvider#stores()
collapsetrue
public <T> List<T> stores(final String storeName, QueryableStoreType<T> type) {
    final List<T> allStores = new ArrayList<>();
    for (StateStoreProvider provider : storeProviders) {
        final List<T> stores =
            provider.stores(storeName, type);
        allStores.addAll(stores);
    }
    if (allStores.isEmpty()) {
        // Replace with StateStoreMigratedException
        throw new InvalidStateStoreException("the state store, " + storeName + ", may have migrated to another instance.");
    }
    return allStores;
}
Code Block
languagejava
titleAbstractStateStore#validateStoreOpen()
collapsetrue
void validateStoreOpen() {
    if (!innerState.isOpen()) {
        // TODO: Replace with StateStoreClosedException
        throw new InvalidStateStoreException("Store " + innerState.name() + " is currently closed.");
    }
}
Code Block
languagejava
titleRocksDBStore#validateStoreOpen()
collapsetrue
private void validateStoreOpen() {
    if (!open) {
        // TODO: Replace with StateStoreClosedException
        throw new InvalidStateStoreException("Store " + this.name + " is currently closed");
    }
}
Code Block
languagejava
titleRocksDBIterator#hasNext()
collapsetrue
public synchronized boolean hasNext() {
    if (!open) {
        // TODO: Replace with StateStoreClosedException
        throw new InvalidStateStoreException(String.format("RocksDB store %s has closed", storeName));
    }

    return iter.isValid();
}
Code Block
languagejava
titleDelegatingPeekingKeyValueIterator#hasNext()
collapsetrue
public synchronized boolean hasNext() {
    if (!open) {
        // TODO: Replace with StateStoreClosedException
        throw new InvalidStateStoreException(String.format("Store %s has closed", storeName));
    }
    if (next != null) {
        return true;
    }

    if (!underlying.hasNext()) {
        return false;
    }

    next = underlying.next();
    return true;
}

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

G
M
T

 

         
 












Text-to-speech function is limited to 200 characters