KAFKA-3452 Follow-up: Optimize ByteStore Scenarios#2333
KAFKA-3452 Follow-up: Optimize ByteStore Scenarios#2333guozhangwang wants to merge 20 commits intoapache:trunkfrom
Conversation
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
…f logged stores beforehand
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
@dguy @mjsax for reviews (this is still WIP):
Some more thoughts
|
| InMemoryWrapperFactory<K, V> disableLogging(); | ||
| } | ||
|
|
||
| private static abstract class AbstractInMemoryFactory<K, V> implements InMemoryWrapperFactory<K, V> { |
There was a problem hiding this comment.
I'm not sure this is necessary? The only sub-class is InMemoryKeyValueFactory.
| StateStoreSupplier build(); | ||
| PersistentWrapperFactory<K, V> disableCaching(); | ||
| } | ||
|
|
There was a problem hiding this comment.
I don't think this should be an inheritance hierarchy. This class seems like it should be a standalone class that is used to collect the decisions/conifg etc for logging and caching. It could be constructed and passed in to what are currently its subclasses.
With inheritance we end up with competing interfaces in the hierarchy, i.e., the same methods on PersistentWrapperFactory and PersistentKeyValueFactory
| this.valueSerde = valueSerde; | ||
| } | ||
|
|
||
| @Override |
There was a problem hiding this comment.
This is an example of how the inheritance doesn't really work. It actually needs to return this. I think it would be much better to get rid of the inheritance and use delegation instead.
|
@guozhangwang - i've made a quick pass to try and understand where this is heading. I left a couple of comments, but i largely decided not to comment too much as this is a WIP and i wasn't sure what you are after. Anyway, i think the refactoring of the In |
|
Something i forgot to add. In It isn't clear to me where you intend for this logic to go now? |
|
Another thing. Point 6 you mention that we can get rid of |
|
About wappers in About About @dguy BTW I think my refactoring is mingled with too many independent features and each of the above refactoring can be done in a single PR so we can make the reviewing process easier. |
|
w.r.t the |
I think you are right. As long as we do not have the full decision on which stores should always / never be caching enabled, we cannot drop it. I guess we can only revisit it if we consider reverting the unification of caching and dedupping. |
|
@guozhangwang - i've been trying to work with this, but it is an uphill battle. There are changes in here that definitely need a KIP, i.e, the refactoring in |
This PR is extracted from #2333 as an incremental fix to ease the reviewing: 1. Removed `storeToProcessorNodeMap` from ProcessorTopology since it was previously used to set the context current record, and can now be replaced with the dirty entry in the named cache. 2. Replaced `sourceStoreToSourceTopic` from ProcessorTopology with `storeToChangelogTopic` map, which includes the corresponding changelog topic name for all stores that are changelog enabled. 3. Modified `ProcessorStateManager` to rely on `sourceStoreToSourceTopic` when retrieving the changelog topic; this makes the second parameter `loggingEnabled` in `register` not needed any more, and we can deprecate the old API with a new one. 4. Also fixed a minor issue in `KStreamBuilder`: if the storeName is not provided in the `table(..)` function, do not create the underlying materialized store. Modified the unit tests to cover this case. 5. Fixed a bunch of other unit tests failures that are exposed by this refactoring, in which we are not setting the applicationId correctly when constructing the mocking processor topology. Author: Guozhang Wang <wangguoz@gmail.com> Reviewers: Damian Guy, Matthias J. Sax, Ewen Cheslack-Postava Closes #2338 from guozhangwang/KMinor-refactor-state-to-changelogtopic
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
@dguy ready for review again. Some changes:
|
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
dguy
left a comment
There was a problem hiding this comment.
Overall LGTM. I left a couple of very minor nit picks.
Also, should we add tests for WrappedWindowStoreIterator and WrappedSessionStoreIterator ?
| return ByteBuffer.wrap(binaryKey).getLong(binaryKey.length - TIMESTAMP_SIZE); | ||
| } | ||
|
|
||
| public static Window extractWindow(final byte [] binaryKey) { |
There was a problem hiding this comment.
Thanks for adding the test. One nit: looks like this method doesn't need to be public.
| @Override | ||
| public void remove(final Windowed<K> sessionKey) { | ||
| validateStoreOpen(); | ||
|
|
There was a problem hiding this comment.
nit: any reason for the extra line?
There was a problem hiding this comment.
re One nit: looks like this method doesn't need to be public. Actually the test needs it to be package-private, and while on that, I feel we can either make it pure private and remove the corresponding test, or leave it public with the test if we speculate it to be used in the future.
I'd go with the second option if you do not have a strong opinion.
| private static final long DEFAULT_CACHE_SIZE_BYTES = 1 * 1024 * 1024L; | ||
| private ThreadCache cache; | ||
| private final Segments segments = new Segments(windowName, retentionPeriod, numSegments); | ||
| private final StateSerdes<Integer, String> serdes = new StateSerdes<>("", Serdes.Integer(), Serdes.String()); |
There was a problem hiding this comment.
Nice tidy up of this test class :-)
I tried to add some tests, but while doing that I found all the logic is covered in corresponding unit tests of |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
| @Override | ||
| public void remove() { | ||
| throw new UnsupportedOperationException("remove not supported"); | ||
| throw new UnsupportedOperationException("remove() is not supported in DelegatingPeekingKeyValueIterator."); |
There was a problem hiding this comment.
Nit: "... DelegatingPeekingKeyValueIterator" -> "..." + getClass().getName()
| void initInternal(final ProcessorContext context) { | ||
| private void initInternal(final ProcessorContext context) { | ||
| this.context = (InternalProcessorContext) context; | ||
| this.serdes = new StateSerdes<>(underlying.name(), |
There was a problem hiding this comment.
Nit: remove this
(we should avoid this whenever possible)
There was a problem hiding this comment.
I'd prefer to leave it here for better code reading.
| this.map = new TreeMap<>(); | ||
| } | ||
|
|
||
| InMemoryKeyValueStore(String name) { |
There was a problem hiding this comment.
add final (also all other methods below)
There was a problem hiding this comment.
KeyValueStore interface does not mark all the parameters as final, and I think it is safer to keep it as is. As for this constructor, will remove it.
|
|
||
| @Override | ||
| public void close() { | ||
| this.open = false; |
|
|
||
| @Override | ||
| public void close() { | ||
| // do nothing |
There was a problem hiding this comment.
May set iter = null to ensure this iterator is not use after closing anymore?
There was a problem hiding this comment.
This iterator is always wrapped in DelegatingPeekingKeyValueIterator, which already does the isOpen validation.
|
|
||
| @Override | ||
| public KeyValue<Long, byte[]> next() { | ||
| if (!bytesIterator.hasNext()) { |
There was a problem hiding this comment.
Do we need this? Should bytesIterator.next() not throw the same exception for this case?
There was a problem hiding this comment.
Since the underlying iterator could be provided by users which does not validate in its own next() call, it's better doing such validation in the higher-level where the library controls.
|
|
||
| @Override | ||
| public void remove() { | ||
| throw new UnsupportedOperationException("remove() is not supported in WrappedWindowStoreIterator."); |
| final KeyValueIterator<String, String> range = store.range(hello, "zooom"); | ||
| assertThat(range.next(), equalTo(KeyValue.pair(hello, world))); | ||
| assertThat(range.next(), equalTo(KeyValue.pair(hi, there))); | ||
| assertFalse(range.hasNext()); |
There was a problem hiding this comment.
I am confused. If we change store.range(hello, "zooom"); should hasNext() not return true (start and end are inclusive). Thus, to me it seems the test is wrong and there must be a bug in the iterator code.
| @Override | ||
| public Long peekNextKey() { | ||
| throw new UnsupportedOperationException("peekNextKey not supported in stub"); | ||
| throw new UnsupportedOperationException("peekNextKey() not supported in ReadOnlyWindowStoreStub."); |
| public void setUp() { | ||
| cache = new ThreadCache("testCache", DEFAULT_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); | ||
| baseDir = TestUtils.tempDirectory("test"); | ||
| context = new MockProcessorContext(baseDir, Serdes.ByteArray(), Serdes.ByteArray(), recordCollector, cache); |
There was a problem hiding this comment.
Can't we initialize both vars directly and remove this method?
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
|
Refer to this link for build results (access rights to CI server needed): |
| store.put(hi, there); | ||
| store.put("zooom", "home"); | ||
| final KeyValueIterator<String, String> range = store.range(hello, "zooom"); | ||
| final KeyValueIterator<String, String> range = store.range(hello, hi); |
There was a problem hiding this comment.
I am still puzzled why the test did not fail with "zooom" as end range. Can you explain? \cc @dguy WDYT?
There was a problem hiding this comment.
Because in the old code of InMemoryKeyValueStore, our range function is implemented as subMap(from, true, to, false) while I changed it to (from, true, to, true) in the latest commit.
|
Thanks for the patch. LGTM |
|
Merged to trunk. |
This PR is extracted from apache#2333 as an incremental fix to ease the reviewing: 1. Removed `storeToProcessorNodeMap` from ProcessorTopology since it was previously used to set the context current record, and can now be replaced with the dirty entry in the named cache. 2. Replaced `sourceStoreToSourceTopic` from ProcessorTopology with `storeToChangelogTopic` map, which includes the corresponding changelog topic name for all stores that are changelog enabled. 3. Modified `ProcessorStateManager` to rely on `sourceStoreToSourceTopic` when retrieving the changelog topic; this makes the second parameter `loggingEnabled` in `register` not needed any more, and we can deprecate the old API with a new one. 4. Also fixed a minor issue in `KStreamBuilder`: if the storeName is not provided in the `table(..)` function, do not create the underlying materialized store. Modified the unit tests to cover this case. 5. Fixed a bunch of other unit tests failures that are exposed by this refactoring, in which we are not setting the applicationId correctly when constructing the mocking processor topology. Author: Guozhang Wang <wangguoz@gmail.com> Reviewers: Damian Guy, Matthias J. Sax, Ewen Cheslack-Postava Closes apache#2338 from guozhangwang/KMinor-refactor-state-to-changelogtopic
This is a refactoring follow-up of apache#2166. Main refactoring changes: 1. Extract `InMemoryKeyValueStore` out of `InMemoryKeyValueStoreSupplier` and remove its duplicates in test package. 2. Add two abstract classes `AbstractKeyValueIterator` and `AbstractKeyValueStore` to collapse common functional logics. 3. Added specialized `BytesXXStore` to accommodate cases where key value types are Bytes / byte[] so that we can save calling the dummy serdes. 4. Make the key type in `ThreadCache` from byte[] to Bytes, as SessionStore / WindowStore's result serialized bytes are in the form of Bytes anyways, so that we can save unnecessary `Bytes.get()` and `Bytes.wrap(bytes)`. Each of these should arguably be a separate PR and I apologize for the mess, this is because this branch was extracted from a rather large diff that has multiple refactoring mingled together and dguy and myself have already put lots of efforts to break it down to a few separate PRs, and this is the only left-over work. Such PR won't happen in the future. Ping dguy enothereska mjsax for reviews Author: Guozhang Wang <wangguoz@gmail.com> Reviewers: Damian Guy, Matthias J. Sax, Jun Rao Closes apache#2333 from guozhangwang/K3452-followup-state-store-refactor
This PR is extracted from apache/kafka#2333 as an incremental fix to ease the reviewing: 1. Removed `storeToProcessorNodeMap` from ProcessorTopology since it was previously used to set the context current record, and can now be replaced with the dirty entry in the named cache. 2. Replaced `sourceStoreToSourceTopic` from ProcessorTopology with `storeToChangelogTopic` map, which includes the corresponding changelog topic name for all stores that are changelog enabled. 3. Modified `ProcessorStateManager` to rely on `sourceStoreToSourceTopic` when retrieving the changelog topic; this makes the second parameter `loggingEnabled` in `register` not needed any more, and we can deprecate the old API with a new one. 4. Also fixed a minor issue in `KStreamBuilder`: if the storeName is not provided in the `table(..)` function, do not create the underlying materialized store. Modified the unit tests to cover this case. 5. Fixed a bunch of other unit tests failures that are exposed by this refactoring, in which we are not setting the applicationId correctly when constructing the mocking processor topology. Author: Guozhang Wang <wangguoz@gmail.com> Reviewers: Damian Guy, Matthias J. Sax, Ewen Cheslack-Postava Closes #2338 from guozhangwang/KMinor-refactor-state-to-changelogtopic
This is a refactoring follow-up of #2166. Main refactoring changes:
Extract
InMemoryKeyValueStoreout ofInMemoryKeyValueStoreSupplierand remove its duplicates in test package.Add two abstract classes
AbstractKeyValueIteratorandAbstractKeyValueStoreto collapse common functional logics.Added specialized
BytesXXStoreto accommodate cases where key value types are Bytes / byte[] so that we can save calling the dummy serdes.Make the key type in
ThreadCachefrom byte[] to Bytes, as SessionStore / WindowStore's result serialized bytes are in the form of Bytes anyways, so that we can save unnecessaryBytes.get()andBytes.wrap(bytes).Each of these should arguably be a separate PR and I apologize for the mess, this is because this branch was extracted from a rather large diff that has multiple refactoring mingled together and @dguy and myself have already put lots of efforts to break it down to a few separate PRs, and this is the only left-over work. Such PR won't happen in the future.
Ping @dguy @enothereska @mjsax for reviews