Skip to content
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-15774: introduce internal StoreFactory #14659

Merged
merged 6 commits into from Nov 7, 2023

Conversation

agavra
Copy link
Contributor

@agavra agavra commented Oct 27, 2023

Overview

This PR sets up the necessary prerequisites to respect configurations such as dsl.default.store.type and the dsl.store.suppliers.class (introduced in #14648) without requiring them to be passed in to StreamBuilder#new(TopologyConfig) (passing them only into new KafkaStreams(...).

It essentially makes StoreBuilder an external-only API and internally it uses the StoreFactory equivalent. It replaces KeyValueStoreMaterializer with an implementation of StoreFactory that creates the store builder only after configure() is called (in a Future PR we will create the missing equivalents for all of the places where the same thing happens for window stores, such as in all the *WindowKStreamImpl classes)

Testing

There is no change in functionality for this PR

Review Guide

  1. Start with looking at StoreFactory and read the JavaDocs, this is an interface representing what used to be InternalTopologyBuilder.StateStoreFactory
  2. Look at how StoreBuilderWrapper is what used to be the implementation of InternalTopologyBuilder.StateStoreFactory
  3. Note that InternalTopologyBuilder#addStateStore now takes in a StoreFactory instead of a StoreBuilder from everywhere that uses the DSL.
  4. The rest is piping that change around and wrapping StoreBuilder with StoreBuilderWrapper. In a future PR all StoreBuilderWrappers in the DSL will be replaced with a new one that respects the configurations passed in to new KafkaStreams

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@ableegoldman ableegoldman changed the title refactor: introduce internal StoreFactory DO NOT MERGE: introduce internal StoreFactory Oct 31, 2023
@@ -520,7 +521,7 @@ public synchronized <K, V> GlobalKTable<K, V> globalTable(final String topic,
*/
public synchronized StreamsBuilder addStateStore(final StoreBuilder<?> builder) {
Objects.requireNonNull(builder, "builder can't be null");
internalStreamsBuilder.addStateStore(builder);
internalStreamsBuilder.addStateStore(new StoreBuilderWrapper<>(builder));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right off the bat I don't love the name StoreBuilderWrapper...but I think we both know this was going to be a practice in painful out of control java naming lol

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I originally tried DelegatingStoreFactory but settled on this. Open to suggestions :)

}
}
nodeGroups = null;
}

public final <KIn, VIn> void addGlobalStore(final StoreBuilder<?> storeBuilder,
public final <KIn, VIn> void addGlobalStore(final StoreFactory<?> storeBuilder,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: change the parameter name to storeFactory

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yup, there were lots of places I need to make this change but I didn't want to touch 2 million lines of code until the general strategy was looked at 😆

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

very understandable -- just wanted to leave a comment in case you missed it/to help identify some of those two million lines 😭

}

@Override
public boolean isCompatibleWith(final StoreFactory<?> storeFactory) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know you didn't add this code (even if the method itself is new) but some javadocs or a more descriptive name might be useful here. Hard to understand whether this is/will be working correctly after the changes when it's unclear what its original semantics/purpose were

I guess this is related to a fundamental source of confusion I have right now, which is: where does the StoreBuilderWrapper fit into the final plan? Will anything extend it, or will uninitialized/unconfigured StoreFactory implementations somehow be compiled into this class when they finally set up a real StoreBuilder? Like is this class a "storebuilder-wrapper" in the sense that it always wraps an actual configured/initialized StoreBuilder object, or will the builder field of this class be "delayed" (in which case how do we evaluate if it's compatible, hence how it all relates to this specific question/method)

Copy link
Contributor Author

@agavra agavra Oct 31, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll add a comment. The plan is to add a separate DelayedStoreFactory which delays the creation of the StoreBuilder until configure is called. the isCompatibleWith method will check to make sure the inner components would generate the same store builder (e.g. the wrapped StoreSuppliers are identical)

I'll just make that change and add it to this PR to show how it works end-to-end, but that will be a functionality change.

EDIT: actually I can add it w/o changing functionality if I just ignore configure()

@agavra agavra changed the title DO NOT MERGE: introduce internal StoreFactory KAFKA-15774: introduce internal StoreFactory Nov 2, 2023
@agavra agavra marked this pull request as ready for review November 2, 2023 23:59
Copy link
Contributor

@ableegoldman ableegoldman left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just the one small comment for now...looks good on the whole. I assume a similar approach will be taken for window/session stores, though we'll have to implement the Materializer class for each of these since they don't already exist

* <li>{@link org.apache.kafka.streams.state.StoreSupplier} is used by the
* DSL to provide preconfigured state stores as well as type-safe stores
* (e.g. {@link org.apache.kafka.streams.state.KeyValueBytesStoreSupplier}.
* Before passing the {@code StoreSupplier} into the </li>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

incomplete sentence?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🙄 🤭 😬

import org.apache.kafka.streams.state.internals.VersionedKeyValueStoreBuilder;
import org.apache.kafka.streams.state.internals.WindowStoreBuilder;

public class StoreBuilderWrapper implements StoreFactory {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok one more thing: the javadocs explaining the StoreFactory are great, but I think this needs javaodcs as well. Is it just the default implementation of the StoreFactory class? Is it just here to cover the window/session store cases because they don't yet have a Materializer? etc

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's here mostly to cover use cases where the user passes in StoreBuilder directly (via PAPI or DSL#processes funs) - so it's here to stay :)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Aha -- thanks for clarifying. So IIUC it is like a "wrapper" for already-instantiated StoreBuilders? The current name actually makes total sense once I understood what the class does. Of course ideally the reverse is true...that you can understand what the class does based on the name. But maybe it's obvious and I just lack reading comprehension skills 🤷‍♀️

I can't think of a better name, but I feel better now that there's a description for this. Thanks!

return StoreType.IN_MEMORY;
case StreamsConfig.ROCKS_DB:
default:
// for backwards compatibility, we ignore invalid store
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Honestly I would just make this change now, and throw an exception if it's invalid. If you really want to keep the behavior the same for whatever reason, that's fine too, but we should at least log a warning

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👌

@agavra
Copy link
Contributor Author

agavra commented Nov 6, 2023

@ableegoldman none of the failing tests seem remotely related and a quick spotcheck shows them passing locally:
image

Copy link
Contributor

@ableegoldman ableegoldman left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@ableegoldman ableegoldman merged commit febf0fb into apache:trunk Nov 7, 2023
1 check failed
@@ -16,30 +16,9 @@
*/
package org.apache.kafka.streams.processor.internals;

import org.apache.kafka.clients.consumer.OffsetResetStrategy;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please avoid auto-reformatting.

@agavra agavra deleted the store_factory branch November 8, 2023 18:45
ableegoldman pushed a commit that referenced this pull request Nov 11, 2023
This is a follow up from #14659 that ports the windowed classes to use the StoreFactory abstraction as well. There's a side benefit of not duplicating the materialization code twice for each StreamImpl/CogroupedStreamImpl class as well.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
rreddy-22 pushed a commit to rreddy-22/kafka-rreddy that referenced this pull request Jan 2, 2024
This PR sets up the necessary prerequisites to respect configurations such as dsl.default.store.type and the dsl.store.suppliers.class (introduced in apache#14648) without requiring them to be passed in to StreamBuilder#new(TopologyConfig) (passing them only into new KafkaStreams(...).

It essentially makes StoreBuilder an external-only API and internally it uses the StoreFactory equivalent. It replaces KeyValueStoreMaterializer with an implementation of StoreFactory that creates the store builder only after configure() is called (in a Future PR we will create the missing equivalents for all of the places where the same thing happens for window stores, such as in all the *WindowKStreamImpl classes)

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
rreddy-22 pushed a commit to rreddy-22/kafka-rreddy that referenced this pull request Jan 2, 2024
This is a follow up from apache#14659 that ports the windowed classes to use the StoreFactory abstraction as well. There's a side benefit of not duplicating the materialization code twice for each StreamImpl/CogroupedStreamImpl class as well.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
anurag-harness pushed a commit to anurag-harness/kafka that referenced this pull request Feb 9, 2024
This PR sets up the necessary prerequisites to respect configurations such as dsl.default.store.type and the dsl.store.suppliers.class (introduced in apache#14648) without requiring them to be passed in to StreamBuilder#new(TopologyConfig) (passing them only into new KafkaStreams(...).

It essentially makes StoreBuilder an external-only API and internally it uses the StoreFactory equivalent. It replaces KeyValueStoreMaterializer with an implementation of StoreFactory that creates the store builder only after configure() is called (in a Future PR we will create the missing equivalents for all of the places where the same thing happens for window stores, such as in all the *WindowKStreamImpl classes)

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
anurag-harness added a commit to anurag-harness/kafka that referenced this pull request Feb 9, 2024
This PR sets up the necessary prerequisites to respect configurations such as dsl.default.store.type and the dsl.store.suppliers.class (introduced in apache#14648) without requiring them to be passed in to StreamBuilder#new(TopologyConfig) (passing them only into new KafkaStreams(...).

It essentially makes StoreBuilder an external-only API and internally it uses the StoreFactory equivalent. It replaces KeyValueStoreMaterializer with an implementation of StoreFactory that creates the store builder only after configure() is called (in a Future PR we will create the missing equivalents for all of the places where the same thing happens for window stores, such as in all the *WindowKStreamImpl classes)

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>

Co-authored-by: Almog Gavra <almog.gavra@gmail.com>
anurag-harness pushed a commit to anurag-harness/kafka that referenced this pull request Feb 9, 2024
This is a follow up from apache#14659 that ports the windowed classes to use the StoreFactory abstraction as well. There's a side benefit of not duplicating the materialization code twice for each StreamImpl/CogroupedStreamImpl class as well.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
anurag-harness added a commit to anurag-harness/kafka that referenced this pull request Feb 9, 2024
…) (#233)

This is a follow up from apache#14659 that ports the windowed classes to use the StoreFactory abstraction as well. There's a side benefit of not duplicating the materialization code twice for each StreamImpl/CogroupedStreamImpl class as well.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>

Co-authored-by: Almog Gavra <almog.gavra@gmail.com>
yyu1993 pushed a commit to yyu1993/kafka that referenced this pull request Feb 15, 2024
This PR sets up the necessary prerequisites to respect configurations such as dsl.default.store.type and the dsl.store.suppliers.class (introduced in apache#14648) without requiring them to be passed in to StreamBuilder#new(TopologyConfig) (passing them only into new KafkaStreams(...).

It essentially makes StoreBuilder an external-only API and internally it uses the StoreFactory equivalent. It replaces KeyValueStoreMaterializer with an implementation of StoreFactory that creates the store builder only after configure() is called (in a Future PR we will create the missing equivalents for all of the places where the same thing happens for window stores, such as in all the *WindowKStreamImpl classes)

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
yyu1993 pushed a commit to yyu1993/kafka that referenced this pull request Feb 15, 2024
This is a follow up from apache#14659 that ports the windowed classes to use the StoreFactory abstraction as well. There's a side benefit of not duplicating the materialization code twice for each StreamImpl/CogroupedStreamImpl class as well.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
AnatolyPopov pushed a commit to aiven/kafka that referenced this pull request Feb 16, 2024
This PR sets up the necessary prerequisites to respect configurations such as dsl.default.store.type and the dsl.store.suppliers.class (introduced in apache#14648) without requiring them to be passed in to StreamBuilder#new(TopologyConfig) (passing them only into new KafkaStreams(...).

It essentially makes StoreBuilder an external-only API and internally it uses the StoreFactory equivalent. It replaces KeyValueStoreMaterializer with an implementation of StoreFactory that creates the store builder only after configure() is called (in a Future PR we will create the missing equivalents for all of the places where the same thing happens for window stores, such as in all the *WindowKStreamImpl classes)

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
AnatolyPopov pushed a commit to aiven/kafka that referenced this pull request Feb 16, 2024
This is a follow up from apache#14659 that ports the windowed classes to use the StoreFactory abstraction as well. There's a side benefit of not duplicating the materialization code twice for each StreamImpl/CogroupedStreamImpl class as well.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
@mjsax mjsax added streams kip Requires or implements a KIP labels Feb 21, 2024
clolov pushed a commit to clolov/kafka that referenced this pull request Apr 5, 2024
This is a follow up from apache#14659 that ports the windowed classes to use the StoreFactory abstraction as well. There's a side benefit of not duplicating the materialization code twice for each StreamImpl/CogroupedStreamImpl class as well.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias Sax <mjsax@apache.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
kip Requires or implements a KIP streams
Projects
None yet
3 participants