Versions Compared

Key

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


(Working in Progress)

 

 

Table of Contents

Status

Current stateUnder Discussion

...

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 InvalidStateStoreExceptions 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
# Two category exceptions
public class StateStoreClosedExceptionRetryableStateStoreException extends InvalidStateStoreException
public class StateStoreMigratedExceptionFatalStateStoreException extends InvalidStateStoreException


# Retryable exceptions
public class StateStoreRetryableExceptionStreamThreadNotStartedException extends RetryableStateStoreException
public class StreamThreadRebalancingException extends RetryableStateStoreException
public class StateStoreMigratedException extends InvalidStateStoreExceptionRetryableStateStoreException


# Fatal exceptions
public class StateStoreFailExceptionStreamThreadNotRunningException extends InvalidStateStoreException

Three categories exception throw to user

FatalStateStoreException

Various state store exceptions can classify into two category exceptions: RetryableStateStoreException and FatalStateStoreException. The user can use the two exceptions if they only need to distinguish whether it can retry.

  • Retryable exceptions
    • StreamThreadNotStartedException: will be thrown when streams thread state is CREATED, the user can retry until to RUNNING.
    • StreamThreadRebalancingException: will be thrown when stream thread is not running and stream state is REBALANCING
    StateStoreRetriableException: 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 need to rediscover the store.
  • StateStoreFailException: Fatal error when access state store, the user cannot retry or rediscover.

...

    • will be thrown when state store already closed and stream state is REBALANCING.
  • Fatal exceptions
    • KafkaStreamsNotRunningException: will be thrown when stream thread is not running and stream state is PENDING_SHUTDOWN / NOT_RUNNING / ERROR. The user cannot retry when this exception is thrown.
    • StateStoreNotAvailableException: will be thrown when state store closed and stream thread 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 following is the public method methods that users will call to get state store instance:

  • KafkaStreams
    • storesstore(storeName, queryableStoreType)
Info
Throw exceptions: StreamThreadNotStartedException, StreamThreadRebalancingException, KafkaStreamsNotRunningExceptionUnknownStateStoreException


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

  • interface ReadOnlyKeyValueStore(class CompositeReadOnlyKeyValueStore)
    • get(kkey)
    • range(from, to)
    • all()
    • approximateNumEntries()
  • ReadOnlySessionStoreinterface ReadOnlySessionStore(CompositeReadOnlySessionStoreclass CompositeReadOnlySessionStore)
    • fetch(kkey)
    • fetch(from, to)
  • ReadOnlyWindowStoreinterface ReadOnlyWindowStore(CompositeReadOnlyWindowStoreclass CompositeReadOnlyWindowStore)
    • fetch(kkey, 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)
    • time)
    • fetch(key, from, to)

    • fetch(from, to, fromTime, toTime)
    • 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: 

StreamThreadRebalancingExceptionStateStoreMigratedExceptionKafkaStreamsNotRunningExceptionStateStoreNotAvailableException


Compatibility, Deprecation, and Migration Plan

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

Rejected Alternatives

None.


G
M
T












  • CompositeReadOnlySessionStore#fetch(key) (v)
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
    • MeteredSessionStore#fetch(key)
      • 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
    • MeteredWindowedKeyValueIterator#hasNext()
      • MergedSortedCacheSessionStoreIterator#hasNext()
        AbstraceMergedSortedCacheStoreIterator#hasNext()
        • FilteredCacheIterator#hasNext()
        • WrappedSessionStoreIterator#hasNext()
          • SegmentIterator#hasNext()
            • Segment.range(from, to)
              RocksDBStore.range(from, to)
              • RocksDBStore.validateStoreOpen() (v)
              • return new RocksDBRangeIterator()
            • return
          • return
        • return
      • return iterator(MeteredWindowedKeyValueIterator)
  • MeteredWindowedKeyValueIterator#next()
    • MergedSortedCacheSessionStoreIterator#next()
      AbstractMergedSortedCacheStoreIterator#next()
      • AbstractMergedSortedCacheStoreIterator#hasNext()
      • FilteredCacheIterator#hasNext()
      • WrappedSessionStoreIterator#hasNext()
        • SegmentIterator#hasNext()
          • Segment.range(from, to)
            RocksDBStore.range(from, to)
            • RocksDBStore.validateStoreOpen() (v)
            • return new RocksDBRangeIterator()
          • return
        • return
      • MergedSortedCacheSessionStoreIterator#nextStoreValue()
        AbstractMergedSortedCacheStoreIterator#nextStoreValue()
        • WrappedSessionStoreIterator#next()
          • SegmentIterator#next()
            • RocksDBRangeIterator#next()
              RocksDbIterator#next()
              • RocksDbIterator#getKeyValue()
              • RocksIterator#next()
              • return entry
            • return
          • return
        • return
      • return
    • return

...

Expand
titleCall trace 7: ReadOnlySessionStore#fetch(from, to)
  • CompositeReadOnlySessionStore#fetch(from, to) (v)
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
    • return new DelegatingPeekingKeyValueIterator<>()
  • DelegatingPeekingKeyValueIterator#hasNext() (v)
    • CompositKeyValueIterator#hasNext()
      • NextIteratorFunction#apply(store)
        • MeteredSessionStore#fetch(from, to)
          • MeteredSessionStore#findSession(from, to, 0, t)
            • CachingSessionStore#findSession(from, to, 0, t)
              • 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()
            • return MeteredWindowedKeyValueIterator()
          • return
        • return
      • MeteredWindowedKeyValueIterator#hasNext()
        • MergedSortedCacheSessionStoreIterator#hasNext()
          AbstractMergedSortedCacheStoreIterator#hasNext()
          • FilteredCacheIterator#hasNext()
          • WrappedSessionStoreIterator#hasNext()
            • SegmentIterator#hasNext()
              • Segment.range(from, to)
                RocksDBStore.range(from, to)
                • RocksDBStore.validateStoreOpen() (v)
                • return new RocksDBRangeIterator()
              • return
            • return
          • return
        • return
      • return
    • CompositeKeyValueIterator#next()
    • return
  • DelegatingPeekingKeyValueIterator#next()
    • return keyvalue

 

 

 

Changed Classes

Code Block
languagejava
public interface QueryableStoreType<T> {
    T create(final KafkaStreams streams, final StateStoreProvider storeProvider, final String storeName);
}
Code Block
languagejava
titleKafkaStreams#store()
collapsetrue
public <T> T store(final String storeName, final QueryableStoreType<T> queryableStoreType) {
    validateIsRunning();
    return queryableStoreProvider.getStore(storeName, queryableStoreType);
}
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()) {
        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()) {
        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()) {
        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()) {
                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()) {
        throw new InvalidStateStoreException("the state store, " + storeName + ", may have migrated to another instance.");
    }
    return allStores;
}
Code Block
languagejava
titleCompositeReadOnlyKeyValueStore
collapsetrue
public V get(final K key) {
    Objects.requireNonNull(key);
    final List<ReadOnlyKeyValueStore<K, V>> stores = storeProvider.stores(storeName, storeType);
    for (ReadOnlyKeyValueStore<K, V> store : stores) {
        try {
            final V result = store.get(key);
            if (result != null) {
                return result;
            }
        } catch (InvalidStateStoreException e) {
            throw new InvalidStateStoreException("State store is not available anymore and may have been migrated to another instance; please re-discover its location from the state metadata.");
        }

    }
    return null;
}
 
public KeyValueIterator<K, V> range(final K from, final K to) {
    Objects.requireNonNull(from);
    Objects.requireNonNull(to);
    final NextIteratorFunction<K, V, ReadOnlyKeyValueStore<K, V>> nextIteratorFunction = new NextIteratorFunction<K, V, ReadOnlyKeyValueStore<K, V>>() {
        @Override
        public KeyValueIterator<K, V> apply(final ReadOnlyKeyValueStore<K, V> store) {
            try {
                return store.range(from, to);
            } catch (InvalidStateStoreException e) {
                throw new InvalidStateStoreException("State store is not available anymore and may have been migrated to another instance; please re-discover its location from the state metadata.");
            }
        }
    };
    final List<ReadOnlyKeyValueStore<K, V>> stores = storeProvider.stores(storeName, storeType);
    return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction));
}

public KeyValueIterator<K, V> all() {
    final NextIteratorFunction<K, V, ReadOnlyKeyValueStore<K, V>> nextIteratorFunction = new NextIteratorFunction<K, V, ReadOnlyKeyValueStore<K, V>>() {
        @Override
        public KeyValueIterator<K, V> apply(final ReadOnlyKeyValueStore<K, V> store) {
            try {
                return store.all();
            } catch (InvalidStateStoreException e) {
                throw new InvalidStateStoreException("State store is not available anymore and may have been migrated to another instance; please re-discover its location from the state metadata.");
            }
        }
    };
    final List<ReadOnlyKeyValueStore<K, V>> stores = storeProvider.stores(storeName, storeType);
    return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>(stores.iterator(), nextIteratorFunction));
}
 
public long approximateNumEntries() {
    final List<ReadOnlyKeyValueStore<K, V>> stores = storeProvider.stores(storeName, storeType);
    long total = 0;
    for (ReadOnlyKeyValueStore<K, V> store : stores) {
        total += store.approximateNumEntries();
        if (total < 0) {
            return Long.MAX_VALUE;
        }
    }
    return total;
}
Code Block
languagejava
titleAbstractStateStore#validateStoreOpen()
collapsetrue
void validateStoreOpen() {
    if (!innerState.isOpen()) {
        throw new InvalidStateStoreException("Store " + innerState.name() + " is currently closed.");
    }
}
Code Block
languagejava
titleRocksDBStore#validateStoreOpen()
collapsetrue
private void validateStoreOpen() {
    if (!open) {
        throw new InvalidStateStoreException("Store " + this.name + " is currently closed");
    }
}
Code Block
languagejava
titleRocksDBIterator#hasNext()
collapsetrue
public synchronized boolean hasNext() {
    if (!open) {
        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) {
        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