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

Compare with Current View Page History

« Previous Version 36 Next »

Status

Current stateUnder Discussion

Discussion thread: here

JIRA KAFKA-5876 - Getting issue details... STATUS

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

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 are add four new exceptions:

public class StateStoreMigratedException extends InvalidStateStoreException
public class StateStoreRetryableException extends InvalidStateStoreException
public class StateStoreFailException extends InvalidStateStoreException
public class StateStoreClosedException extends InvalidStateStoreException

Three categories exception throw to user

  • StateStoreRetriableException: The application instance in the state of rebalancing, the user just need retry and wait until rebalance finished.
  • 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.

One internal exception: StateStoreClosedException.

The following is the public method that users will call:

  • KafkaStreams
    • store()
  • 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()
  • WindowStoreIterator(MeteredWindowStoreIterator)
    • next()
    • hasNext()
    • peekNextKey()
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();
}

class MeteredWindowStoreIterator<V> implements WindowStoreIterator<V> {
    public boolean hasNext();
    public KeyValue<Long, V> next();
    public Long 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_SHUTDOWN or ERROR or NOT_RUNNING:
    • wrap InvalidStateStoreException(include subclass) to StateStoreFailException

Call Trace

 

  • KafkaStreams#store() (v)
    • QueryableStoreProvider#getStore() (v)
      • GlobalStateStoreProvider#stores() (v)
      • StreamThreadStateStroeProvider#stores() (v)
  • 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()
  • 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
  • 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

 

  • 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
  • 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() (v)
                      • 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

 

  • 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
  • CompositeReadOnlySessionStore#fetch(key) (v)
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
    • MeteredWindowStore#fetch()
      • CachingWindowStore#fetch(k, tf, tt)
        • AbstractStateStore#validateStoreOpen()
          • ChangeLoggingWindowBytesStore#isOpen()
            • AbstractStateStore#isOpen()
              • RocksDBWindowBytesStore#isOpen()
                • RocksDBSegmentedBytesStore#isOpen()
                • return
              • return
            • return
          • return
        • ChangeLoggingWindowBytesStore#fetch()
          • RocksDBWindowStoreBytesStore#fetch()
            • RocksDBSegmentedBytesStore#fetch()
              • WindowKeySchema#segmentsToSearch()
                • Segments#segments()
                  • RocksDBStore#isOpen()
                  • return segments
                • return
              • return new SegmentIterator()
            • WindowStoreIteratorWrapper.bytesIterator()
              • return new WrappedWindowStoreBytesIterator()
            • return
          • return
        • ThreadCache#range()
          • return new MemoryLRUCacheBytesIterator
        • return new MergedSortedCacheWindowStoreIterator
      • return new MeteredWindowStoreIterator
    • MeteredWindowStoreIterator#hasNext()
      AbstractMergedSortedCacheStoreIterator#hasNext()
      • WrappedWindowStoreIterator#hasNext()
        • SegmentIterator#hasNext()
          • Segment#range()
            RocksDBStore#range()
            • RocksDBStore#validateStoreOpen()
            • return RocksDBRangeIterator
          • return
        • return
      • return
    • return
  • MeteredWindowStoreIterator#hasNext()
    • MeteredSortedCacheWindowStoreIterator#hasNext()
      AbstractMergedSortedCacheStoreIterator#hasNext()
      • WrappedWindowStoreIterator#hasNext()
        • SegmentIterator#hasNext()
          • Segment#range()
            RocksDBStore#range()
            • RocksDBStore#validateStoreOpen() (v)
            • return RocksDBRangeIterator
          • return
        • return
      • return
    • return
  • MeteredWindowStoreIterator#next()
    • MergedSortedCacheWindowStoreIterator#next()
      AbstractMergedSortedCacheStoreIterator#next()
      • WrappedWindowStoreBytesIterator#hasNext()
        • SegmentIterator#hasNext()
          • return
        • return
      • AbstractMergedSortedCacheStoreIterator#nextStoreValue()
        • WrappedWindowStoreBytesIterator#next()
          • SegmentIterator#next()
            • SegmentIterator#hasNext()
              • return
            • RocksDBRangeIterator#next()
              • RocksDBRangeIterator#hasNext()
                • RocksDBIterator#hasNext() (v)
                • return
              • RocksIterator#next()
              • return entry
            • return
          • return keyvalue
        • return
      • return
    • return keyvalue
  • CompositeReadOnlyWindowStore#fetch(from, to)
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
      • return
    • return new DelegatingPeekingKeyValueIterator()
  • DelegatingPeekingKeyValueIterator#hasNext() (v)
    • CompositeKeyValueIterator#hasNext()
      • NextIteratorFunciton#apply()
        • MeteredWindowStore#fetch()
          • CachingWindowStore#fetch()
            • AbstractStateStore#validateStoreOpen() (v)
            • ChangeLoggingWindowBytesStore#fetch()
              • RocksDBWindowBytesStore#fetch()
                • RocksDBSegmentedBytesStore#fetch()
                  • WindowKeySchema#segmentsToSearch()
                    • Segments#segments() (v)
                      • return
                    • return
                  • return SegmentIterator
                • return keyvalueIterator
              • return
            • ThreadCache.range()
              • return new MemoryLRUCacheBytesIterator
            • WindowKeySchema#hasNextCondition()
            • return new MergedSortedCacheWindowStoreKeyValueIterator()
          • return MeteredWindowKeyValueIterator()
        • return
      • MeteredWindowedKeyValueIterator#hasNext()
        • MergedSortedCacheWindowStoreKeyValueIterator#hasNext()
          AbstractMergedSortedCacheStoreIterator
          • WrappedWindowStoreBytesIterator#hasNext()
            WrappedKeyValueIterator#hasNext()
            • SegementIterator#hasNext()
              • Segment#range()
                RocksDBStore#range()
                • RocksDBStore#validateStoreOpen() (v)
                • return RocksDBRangeIterator
              • return
          • return
        • return
      • return
    • CompositeKeyValueIterator#next()
      • MeteredWindowedKeyValueIterator#next()
        • MergedSortedCacheWindowStoreKeyValueIterator#next()
          AbstractMergedSortedCacheStoreIterator#next()
          • AbstractMergedSortedCacheStoreIterator#hasNext()
          • AbstractMergedSortedCacheStoreIterator#nextStoreValue()
            • WrappedWindowStoreBytesIterator#next()
              • SegmentIteator#hasNext()
              • SegmentIterator#next()
                • SegmentIterator#hasNext() (v)
                • RocksDBRangeIterator#next()
                  RocksDbIterator#next()
                  • RocksDbIterator#hasNext()
                  • RocksIterator#next()
                  • return entry
                • return
              • return keyvalue
            • return
          • return keyvalue
        • return
      • return
    • return
  • DelegatingPeekingKeyValueIterator#next()
    • DelegatingPeekingKeyValueIterator#hasNext()
    • return
  • CompositeReadOnlyWindowStore#all()
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
      • return
    • return new DelegatingPeekingKeyValueIterator()
  • DelegatingPeekingKeyValueIterator#hasNext() (v)
    • CompositeKeyValueIterator#hasNext()
      • NextIteratorFunciton#apply()
        • MeteredWindowStore#all()
          • CachingWindowStore#all()
            • AbstractStateStore#validateStoreOpen() (v)
            • ChangeLoggingWindowBytesStore#all()
              • RocksDBWindowBytesStore#all()
                • RocksDBSegmentedBytesStore#all()
                  • Segments#allSegments() (v)
                  • return new SegmentIterator()
                • return keyValueIterator
              • return
            • ThreadCache#all()
              • return new MemoryLRUCacheBytesIterator
            • return new MergedSortedCacheWindowStoreKeyValueIterator
          • return new MeteredWindowedKeyValueIterator()
        • return
      • MeteredWindowedKeyValueIterator#hasNext()
        • MergedSortedCacheWindowStoreKeyValueIterator#hasNext()
          AbstractMergedSortedCacheStoreIterator#hasNext()
          • WrappedKeyValueIterator#hasNext()
            • SegmentIterator#hasNext() (v)
              • return currentIterator
            • return
          • return
        • return
      • return
    • CompositeKeyValueIterator#next()
      • MeteredWindowedKeyValueIterator#next()
        • MergedSortedCacheWindowStoreKeyValueIterator#next()
          AbstractMergedSortedCacheStoreIterator#next()
          • WrappedKeyValueIterator#hasNext()
            • SegmentIterator#hasNext()
          • AbstractMergedSortedCacheStoreIterator#nextStoreValue()
            • WrappedKeyValueIterator#next()
              • SegmentIterator#next()
                • RocksDbIterator#next()
                  • RocksDbIterator#hasNext() (v)
                  • RocksIterator.next()
                  • return entry
                • return
              • return keyvalue
            • return
          • return
        • return keyvalue
      • return
    • return
  • DelegatingPeekingKeyValueIterator#next()
    • DelegatingPeekingKeyValueIterator#hasNext()
    • return
  • CompositeReadOnlyWindowStore#fetchAll()
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
      • return
    • return new DelegatingPeekingKeyValueIterator()
  • DelegatingPeekingKeyValueIterator#hasNext() (v)
    • CompositeKeyValueIterator#hasNext()
      • NextIteratorFunciton#apply()
        • MeteredWindowStore#fetchAll()
          • CachingWindowStore#fetchAll()
            • AbstractStateStore#validateStoreOpen() (v)
            • ChangeLoggingWindowBytesStore#fetchAll()
              • RocksDBWindowBytesStore#fetchAll()
                • RocksDBSegmentedBytesStore#fetchAll()
                  • Segments#allSegments() (v)
                  • return new SegmentIterator()
                • return keyValueIterator
              • return
            • ThreadCache#all()
              • return new MemoryLRUCacheBytesIterator
            • return new MergedSortedCacheWindowStoreKeyValueIterator
          • return new MeteredWindowedKeyValueIterator()
        • return
      • MeteredWindowedKeyValueIterator#hasNext()
        • MergedSortedCacheWindowStoreKeyValueIterator#hasNext()
          AbstractMergedSortedCacheStoreIterator#hasNext()
          • WrappedKeyValueIterator#hasNext()
            • SegmentIterator#hasNext() (v)
              • return currentIterator
            • return
          • return
        • return
      • return
    • CompositeKeyValueIterator#next()
      • MeteredWindowedKeyValueIterator#next()
        • MergedSortedCacheWindowStoreKeyValueIterator#next()
          AbstractMergedSortedCacheStoreIterator#next()
          • WrappedKeyValueIterator#hasNext()
            • SegmentIterator#hasNext() (v)
            • return
          • AbstractMergedSortedCacheStoreIterator#nextStoreValue()
            • WrappedKeyValueIterator#next()
              • SegmentIterator#next()
                • RocksDbIterator#next()
                  • RocksDbIterator#hasNext() (v)
                  • RocksIterator.next()
                  • return entry
                • return
              • return keyvalue
            • return
          • return
        • return keyvalue
      • return
    • return
  • DelegatingPeekingKeyValueIterator#next()
    • DelegatingPeekingKeyValueIterator#hasNext()
    • return


Changes in call trace

 

public interface QueryableStoreType<T> {
    // TODO: pass stream instance parameter
    T create(final KafkaStreams streams, final StateStoreProvider storeProvider, final String storeName);
}
class QueryableStoreProvider
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);
}
class GlobalStateStoreProvider
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);
}
class StreamThreadStateStoreProvider
 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 StateStoreRetryableException
        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;
}
class WrappingStoreProvider
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()) {
        // TODO: Replace with StateStoreMigratedException
        throw new InvalidStateStoreException("The state store, " + storeName + ", may have migrated to another instance.");
    }
    return allStores;
}
class AbstractStateStore
void validateStoreOpen() {
    if (!innerState.isOpen()) {
        // TODO: Replace with StateStoreClosedException
        throw new InvalidStateStoreException("Store " + innerState.name() + " is currently closed.");
    }
}
class RocksDBStore
private void validateStoreOpen() {
    if (!open) {
        // TODO: Replace with StateStoreClosedException
        throw new InvalidStateStoreException("Store " + this.name + " is currently closed");
    }
}
class RocksDBIterator
public synchronized boolean hasNext() {
    if (!open) {
        // TODO: Replace with StateStoreClosedException
        throw new InvalidStateStoreException(String.format("RocksDB store %s has closed", storeName));
    }

    return iter.isValid();
}
class DelegatingPeekingKeyValueIterator
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;
}
class Segments
List<Segment> segments(final long timeFrom, final long timeTo) {
    final long segFrom = Math.max(minSegmentId, segmentId(Math.max(0L, timeFrom)));
    final long segTo = Math.min(maxSegmentId, segmentId(Math.min(maxSegmentId * segmentInterval, Math.max(0, timeTo))));

    final List<Segment> segments = new ArrayList<>();
    for (long segmentId = segFrom; segmentId <= segTo; segmentId++) {
        Segment segment = getSegment(segmentId);
        if (segment != null && segment.isOpen()) {
            try {
                segments.add(segment);
            } catch (InvalidStateStoreException ise) {  // TODO: Replace with StateStoreClosedException
                // segment may have been closed by streams thread;
            }
        }
    }
    return segments;
}

List<Segment> allSegments() {
    final List<Segment> segments = new ArrayList<>();
    for (Segment segment : this.segments.values()) {
        if (segment.isOpen()) {
            try {
                segments.add(segment);
            } catch (InvalidStateStoreException ise) {  // TODO: Replace with StateStoreClosedException
                // segment may have been closed by streams thread;
            }
        }
    }
    Collections.sort(segments);
    return segments;
}
class SegmentIterator
public boolean hasNext() {
    boolean hasNext = false;
    while ((currentIterator == null || !(hasNext = hasNextCondition.hasNext(currentIterator)) || !currentSegment.isOpen())
            && segments.hasNext()) {
        close();
        currentSegment = segments.next();
        try {
            if (from == null || to == null) {
                currentIterator = currentSegment.all();
            } else {
                currentIterator = currentSegment.range(from, to);
            }
        } catch (InvalidStateStoreException e) {  // TODO: Replace with StateStoreClosedException
            // segment may have been closed so we ignore it.
        }
    }
    return currentIterator != null && hasNext;
}

 

 

 

Compatibility, Deprecation, and Migration Plan

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

Rejected Alternatives

None.


G
M
T

 

         

 

Text-to-speech function is limited to 200 characters

 

  • No labels