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

Compare with Current View Page History

« Previous Version 22 Next »

(Working in Progress)

 

 

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

Three categories exception to user

  • StateStoreRetryableException: just wait, plain retry
  • StateStoreMigratedException: need rediscover
  • StateStoreFailException: fatal error, cannot retry or rediscover

 

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

 

Call Trace

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

 

 

 

Changed Classes

KafkaStreams#store()
public <T> T store(final String storeName, final QueryableStoreType<T> queryableStoreType) {
    validateIsRunning();
    return queryableStoreProvider.getStore(storeName, queryableStoreType);
}
QueryableStoreProvider#getStore()
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);
}
GlobalStateStoreProvider#stores()
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);
}
StreamThreadStateStoreProvider#stores()
 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("the state store, " + storeName + ", may have migrated to another instance.");
    }
    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("the state store, " + storeName + ", may have migrated to another instance.");
            }
            stores.add((T) store);
        }
    }
    return stores;
}
WrappingStoreProvider#stores()
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;
}
CompositeReadOnlyKeyValueStore
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;
}
AbstractStateStore#validateStoreOpen()
void validateStoreOpen() {
    if (!innerState.isOpen()) {
        throw new InvalidStateStoreException("Store " + innerState.name() + " is currently closed.");
    }
}
RocksDBStore#validateStoreOpen()
private void validateStoreOpen() {
    if (!open) {
        throw new InvalidStateStoreException("Store " + this.name + " is currently closed");
    }
}
RocksDBIterator#hasNext()
public synchronized boolean hasNext() {
    if (!open) {
        throw new InvalidStateStoreException(String.format("RocksDB store %s has closed", storeName));
    }

    return iter.isValid();
}
DelegatingPeekingKeyValueIterator#hasNext()
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;
}

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 

 


  • No labels