-
Notifications
You must be signed in to change notification settings - Fork 14.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-6144: option to query restoring and standby #7962
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,17 +23,15 @@ | |
import java.util.ArrayList; | ||
import java.util.List; | ||
|
||
import static java.util.Collections.singletonList; | ||
|
||
/** | ||
* A wrapper over all of the {@link StateStoreProvider}s in a Topology | ||
*/ | ||
public class QueryableStoreProvider { | ||
|
||
private final List<StateStoreProvider> storeProviders; | ||
private final List<StreamThreadStateStoreProvider> storeProviders; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. thinking aloud: even though we are binding to a specific StateStoreProvider implementation here, it seems fine,since there are n't any other really in a topology There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is part of the point of this change, actually, |
||
private final GlobalStateStoreProvider globalStoreProvider; | ||
|
||
public QueryableStoreProvider(final List<StateStoreProvider> storeProviders, | ||
public QueryableStoreProvider(final List<StreamThreadStateStoreProvider> storeProviders, | ||
final GlobalStateStoreProvider globalStateStoreProvider) { | ||
this.storeProviders = new ArrayList<>(storeProviders); | ||
this.globalStoreProvider = globalStateStoreProvider; | ||
|
@@ -45,24 +43,28 @@ public QueryableStoreProvider(final List<StateStoreProvider> storeProviders, | |
* | ||
* @param storeName name of the store | ||
* @param queryableStoreType accept stores passing {@link QueryableStoreType#accepts(StateStore)} | ||
* @param includeStaleStores if true, include standbys and recovering stores; | ||
* if false, only include running actives. | ||
* @param <T> The expected type of the returned store | ||
* @return A composite object that wraps the store instances. | ||
*/ | ||
public <T> T getStore(final String storeName, | ||
final QueryableStoreType<T> queryableStoreType) { | ||
final QueryableStoreType<T> queryableStoreType, | ||
final boolean includeStaleStores) { | ||
final List<T> globalStore = globalStoreProvider.stores(storeName, queryableStoreType); | ||
if (!globalStore.isEmpty()) { | ||
return queryableStoreType.create(new WrappingStoreProvider(singletonList(globalStoreProvider)), storeName); | ||
return queryableStoreType.create(globalStoreProvider, storeName); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. thinking aloud: guess there is nt much value in wrapping a single provider.. so +1 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, the role of the wrapping store provider really is nothing more than just iterating over the list of providers. For global stores, there's always exactly one provider, so it's purely a performance penalty for no gain at all. |
||
} | ||
final List<T> allStores = new ArrayList<>(); | ||
for (final StateStoreProvider storeProvider : storeProviders) { | ||
allStores.addAll(storeProvider.stores(storeName, queryableStoreType)); | ||
for (final StreamThreadStateStoreProvider storeProvider : storeProviders) { | ||
allStores.addAll(storeProvider.stores(storeName, queryableStoreType, includeStaleStores)); | ||
} | ||
if (allStores.isEmpty()) { | ||
throw new InvalidStateStoreException("The state store, " + storeName + ", may have migrated to another instance."); | ||
} | ||
return queryableStoreType.create( | ||
new WrappingStoreProvider(storeProviders), | ||
storeName); | ||
new WrappingStoreProvider(storeProviders, includeStaleStores), | ||
storeName | ||
); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
|
||
import org.apache.kafka.streams.errors.InvalidStateStoreException; | ||
import org.apache.kafka.streams.processor.StateStore; | ||
import org.apache.kafka.streams.processor.TaskId; | ||
import org.apache.kafka.streams.processor.internals.StreamThread; | ||
import org.apache.kafka.streams.processor.internals.Task; | ||
import org.apache.kafka.streams.state.QueryableStoreType; | ||
|
@@ -28,11 +29,9 @@ | |
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
/** | ||
* Wrapper over StreamThread that implements StateStoreProvider | ||
*/ | ||
public class StreamThreadStateStoreProvider implements StateStoreProvider { | ||
public class StreamThreadStateStoreProvider { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. With this change, just Global and Wrapping StateStoreProviders exist? IIUC, most of the Anyway, I am not familiar enough with this part of the code. So I leave it to you.. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I also feel that we can collapse the layered interfaces a bit further after we've changed this, since the original motivation of having it is just to "stitch" the global stores and local stores together when exposing as We can consider that in a separate, cleanup PR afterwards. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No, StateStoreProvider is used in the public API as part of QueryableStoreType. Its role is well defined already. That's the one that we should continue to use "everywhere".
So, if anything, we could increase clarity by renaming some of these classes to more accurately reflect their roles, but all the involved classes have well defined and necessary roles right now. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @guozhangwang I think we wrote our comments at the same time. Just to respond directly, I do think that we can clean up the code base in a follow-on change (by renaming stuff that's named misleadingly now), but I don't think we can collapse the hierarchy at all. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks! SGTM. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I can work on the follow-on change. WrappingStoreProvider -> WrappingStateStoreProvider QueryableStoreProvider is left as is. |
||
|
||
private final StreamThread streamThread; | ||
|
||
|
@@ -41,33 +40,40 @@ public StreamThreadStateStoreProvider(final StreamThread streamThread) { | |
} | ||
|
||
@SuppressWarnings("unchecked") | ||
@Override | ||
public <T> List<T> stores(final String storeName, final QueryableStoreType<T> queryableStoreType) { | ||
public <T> List<T> stores(final String storeName, | ||
final QueryableStoreType<T> queryableStoreType, | ||
final boolean includeStaleStores) { | ||
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 (final 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."); | ||
} | ||
if (store instanceof TimestampedKeyValueStore && queryableStoreType instanceof QueryableStoreTypes.KeyValueStoreType) { | ||
stores.add((T) new ReadOnlyKeyValueStoreFacade((TimestampedKeyValueStore<Object, Object>) store)); | ||
} else if (store instanceof TimestampedWindowStore && queryableStoreType instanceof QueryableStoreTypes.WindowStoreType) { | ||
stores.add((T) new ReadOnlyWindowStoreFacade((TimestampedWindowStore<Object, Object>) store)); | ||
} else { | ||
stores.add((T) store); | ||
final StreamThread.State state = streamThread.state(); | ||
if (includeStaleStores ? state.isAlive() : state == StreamThread.State.RUNNING) { | ||
final Map<TaskId, ? extends Task> tasks = includeStaleStores ? streamThread.allTasks() : streamThread.activeTasks(); | ||
final List<T> stores = new ArrayList<>(); | ||
for (final Task streamTask : 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."); | ||
} | ||
if (store instanceof TimestampedKeyValueStore && queryableStoreType instanceof QueryableStoreTypes.KeyValueStoreType) { | ||
stores.add((T) new ReadOnlyKeyValueStoreFacade<>((TimestampedKeyValueStore<Object, Object>) store)); | ||
} else if (store instanceof TimestampedWindowStore && queryableStoreType instanceof QueryableStoreTypes.WindowStoreType) { | ||
stores.add((T) new ReadOnlyWindowStoreFacade<>((TimestampedWindowStore<Object, Object>) store)); | ||
} else { | ||
stores.add((T) store); | ||
} | ||
} | ||
} | ||
return stores; | ||
} else { | ||
throw new InvalidStateStoreException("Cannot get state store " + storeName + " because the stream thread is " + | ||
state + ", not RUNNING" + | ||
(includeStaleStores ? " or REBALANCING" : "")); | ||
} | ||
return stores; | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
cosmetic: extra space at the start
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I noticed that, too. Oh well.