Versions Compared

Key

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


 

Table of Contents

Status

Current stateUnder Discussion

...

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

Proposed Changes

The following is the public methods that users will call for get state store instance or get store values:

...

  • store()

...

  • get(k)
  • range(from, to)
  • all()
  • approximateNumEntries()

...

  • fetch(k)
  • fetch(from, to)

...

...

  • next()
  • hasNext()
  • peekNextKey()

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 RetryableStateStoreException extends InvalidStateStoreException
public class FatalStateStoreException extends InvalidStateStoreException


# Retryable exceptions
public class StreamThreadNotStartedException extends RetryableStateStoreException
public class StreamThreadRebalancingException extends RetryableStateStoreException
public class StateStoreMigratedException extends RetryableStateStoreException


# Fatal exceptions
public class StreamThreadNotRunningException extends FatalStateStoreException
public class StateStoreNotAvailableException extends FatalStateStoreException


# Internal exceptions
public class EmptyStateStoreException extends InvalidStateStoreException
public class StateStoreClosedException extends InvalidStateStoreException

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
  • StateStoreClosedException: will be thrown when state store is not open. This is an internal exception and will be wrapped in StreamThreadNotStartedException or StateStoreMigratedException or StateStoreNotAvailableException later.
  • EmptyStateStoreException: will be thrown when state store cannot be found in all StateStoreProviders. This is an internal exception, and will be wrapped in StreamThreadNotStartedException or StateStoreMigratedException or StateStoreNotAvailableException later.
  • StreamThreadNotRunningException: 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.
  • StreamThreadNotStartedException
    • : will be thrown when
    stream
    • streams thread
    is not running and stream
    • 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, the user just retry and wait until rebalance finished (RUNNING).
    • StateStoreMigratedException: A wrapper for StateStoreClosedException / EmptyStateStoreException. Will be thrown when stream thread is RUNNING / REBALANCING. The user
    need
    • needs to rediscover the state store.
  • Fatal exceptions
    • StreamThreadNotRunningException: 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: A wrapper for StateStoreClosedException / EmptyStateStoreException. Will be thrown when stream thread is PENDING_SHUTDOWN / NOT_RUNNING / ERROR. The user cannot retry when this exception is thrown.

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.

The interface QueryableStoreType will append a new method:

Code Block
languagejava
public interface QueryableStoreType<T> {
  void setStreams(final KafkaStreams streams);
}

This can be assigned KafkaStreams instance to all QueryableStoreType instances for checking KafkaStreams state.

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)
  • Internal exceptions
    • StateStoreClosedException: will be thrown when state store is not open. This is an internal exception and will be wrapped in StateStoreMigratedException or StateStoreNotAvailableException later.
    • EmptyStateStoreException: will be thrown when state store cannot be found in all StateStoreProviders. This is an internal exception and will be wrapped in StateStoreMigratedException or StateStoreNotAvailableException later.


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

  • KafkaStreams
    • store(storeName, queryableStoreType)
  • interface ReadOnlyKeyValueStore(class CompositeReadOnlyKeyValueStore)
    • get(key)
    • range(from, to)
    • all()
    • approximateNumEntries()
  • interface ReadOnlySessionStore(class CompositeReadOnlySessionStore)
    • fetch(key)
    • fetch(from, to)
  • interface ReadOnlyWindowStore(class CompositeReadOnlyWindowStore)
    • 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()

All the above methods could be throw following exceptions:

  • StreamThreadNotStartedException
  • StreamThreadRebalancingException
  • StateStoreMigratedException
  • StreamThreadNotRunningException
  • StateStoreNotAvailableException


  • 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

...

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
Expand
titleCall trace 8: ReadOnlyWindowStore#fetch(key)
  • 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
Expand
titleCall trace 9: ReadOnlyWindowStore#fetch(from, to)
  • 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
Expand
titleCall trace 10: ReadOnlyWindowStore#all()
  • 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
Expand
titleCall trace 11: ReadOnlyWindowStore#fetchAll()
  • CompositeReadOnlyWindowStore#fetchAll()
    • WrappingStoreProvider#stores() (v)
      • StreamThreadStateStoreProvider#stores() (v)
      • return
    • return new DelegatingPeekingKeyValueIterator()
  • DelegatingPeekingKeyValueIterator#hasNext() (v)
    • CompositeKeyValueIterator#hasNext()
      • NextIteratorFunction#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

Compatibility, Deprecation, and Migration Plan

...

Rejected Alternatives

None.


G
M
T

 

 
        

 












Text-to-speech function is limited to 200 characters