KAFKA-6521: Use timestamped stores for KTables#6667
Conversation
|
Call for early review @guozhangwang @bbejeck @vvcephei @ableegoldman @cadonna @abbccdda |
There was a problem hiding this comment.
Fix bug that was exposed with this PR.
vvcephei
left a comment
There was a problem hiding this comment.
Thanks for the PR, @mjsax . I guess the idea here is to just mix in the new (timestamped) stores without changing any semantics? And then we'll have other PR(s) later to update the semantics and tests? This seems like a good way to break it down.
Just a few comments.
There was a problem hiding this comment.
This logic has come up once or twice ;)
Just a random idea... What do you think about a static method on ValueAndTimestamp:
static <V> V getValueOrNull(final ValueAndTimestamp<V> valueAndTimestamp) {
return valueAndTimestamp == null ? null : valueAndTimestamp.value();
}Then all these other methods could be:
| return valueAndTimestamp == null ? null : valueAndTimestamp.value(); | |
| return getValueOrNull(valueAndTimestamp); |
?
There was a problem hiding this comment.
Ack. I'll just update the KIP accordingly. Turn out, we already have ValueAndTimestamp#make and no public constructor (also not documented on the KIP).
There was a problem hiding this comment.
I found it useful with the FullChangeSerde to make the constructor private and force usage via a null-safe factory method. Then, you never have to worry about this check.
There was a problem hiding this comment.
Seems to be a single occurrence. Maybe not worth a factory. Also, ValueAndTimestampSerde checks for non-null and thus we would surface any issue during testing anyway.
|
Updated this. |
vvcephei
left a comment
There was a problem hiding this comment.
Thanks for the update!
Just another minor comment on ValueAndTimestamp
There was a problem hiding this comment.
Since the only usage of the constructor is in this class, it should be ok to be private.
Also, since it's just a data struct, we could also make the class final.
|
Updated this. |
There was a problem hiding this comment.
We already import the class on L26, so let's remove this import.
There was a problem hiding this comment.
This is for statically use this single util function (otherwise we have to call ValueAndTimestamp.getValueOrNull when calling).
There was a problem hiding this comment.
Is having curly braces in one line a convention?
There was a problem hiding this comment.
I like to have some more line to separate nested classes from method
There was a problem hiding this comment.
We could omit L28-29 since we don't want to comment on it.
There was a problem hiding this comment.
Good eye. It would be even better to document the parameters!
There was a problem hiding this comment.
How about
if (!cachingEnabled) {
context.forward(key, new Change<>(newValue, oldValue));
}
|
#6661 has been merged now. retest this please |
|
retest this please |
guozhangwang
left a comment
There was a problem hiding this comment.
Some minor comments.
There was a problem hiding this comment.
This is for statically use this single util function (otherwise we have to call ValueAndTimestamp.getValueOrNull when calling).
There was a problem hiding this comment.
Today we store the timestamp in cache entry and upon flushing we call entry.entry().context().timestamp() to set the timestamp. Would that be replaced with the underlying store's timestamp then?
There was a problem hiding this comment.
Maybe. After the store refactoring, I realized that we might actually be able to revert #6147 -- however, I would prefer to do this as cleanup only after everything else is merged (to make sure it's really safe).
There was a problem hiding this comment.
While working on a follow PR, I figure out the details. The TimestampedForwardingCacheFlushListener needs to extract the timestamp from the value, and set it as record timestamp. Similar to SessionCacheFlushListener in another PR: https://github.com/apache/kafka/pull/6645/files#diff-4039efe54e3b87d65c20bd7d14c1b149R43
There was a problem hiding this comment.
nit: we can still use the ternary operator here.
There was a problem hiding this comment.
Can we just remove the old CacheFlushListener / TupleForwarder since they are internal classes only used in the DSL?
There was a problem hiding this comment.
We still need the old one, because we don't use TimestampedSessionWindowStore in the DSL but keep SessionWindowStore.
There was a problem hiding this comment.
Can we replace other uses of GenericInMemoryKeyValueStore so we can remove the old class as well?
There was a problem hiding this comment.
Not sure. We don't remove "plain" stores, and thus it seems to make sense to keep existing tests and only rewrite DSL related tests?
7048ea9 to
161d6fc
Compare
|
Updated and rebased this. |
There was a problem hiding this comment.
This fixes a bug that was exposed after rebasing. We need to treat in-memory stores always as TimestampedByteStores to guarantee correct restore. (Similar below for window stores.)
There was a problem hiding this comment.
Added a new test to #6179 that also exposed this bug (cf. StoreUpgradeIntegrationTest)
There was a problem hiding this comment.
I agree with the fix, but it raises another question for me.
I admit I could be either missing something or forgetting some details already spelled out. Why don't we convert the values for in-memory stores to also have timestamps in the value? I'm basing on the fact that the InMemoryTimestampedKeyValueStoreMarker does simple puts/gets.
There was a problem hiding this comment.
The in-memory store will have no content on startup, and the upper layer will put/get byte[] -- the store does not care what is in it. The upper layer will already serialize value+timestamp and put those, and expects value+timestamp byte[] back -- hence, the in-memory store does not need to do any translation.
For persistent store, the store does have content on local disk, namely plain values. Because this store cannot be upgraded, we put the adapter in between, to remove the timestamp part on put() and add dummy timestamp on get(), to meet the contract to the upper layer that expects byte[] to be value+timestamp.
If we restore a persistent store, the just copy plain key/value from the changelog topic and put it into the store. Everything still works as expected.
If we restore an in-memory store, we need to add the timestamp to the value, and this is triggered by the TimestampeByteStore interface. Otherwise, the in-memory store would contain plain values, and violate the contract to the upper layer, because the assumption is, that everything store in it, has value+timestamp format.
Ie, for persistent store, we play the backward compatibility game using surrogate timestamps. However, for in-memory store, we don't need to do this, because there is no local on-disk content and we can switch to timestamped-store behavior without any drawbacks and get full feature set.
Does this make sense?
There was a problem hiding this comment.
"exposed" after rebase -- for time-windows, we want to use the end-timestamp (even if this change is not implemented yet, I added it here, to avoid touching this class later on again).
There was a problem hiding this comment.
The new wrapping would break the test. Using RocksDBTimestampedStore avoid the wrapping as TimestampedByteStore interface is implemented. (Same below.)
161d6fc to
73f05da
Compare
|
Rebased to resolve merge conflicts. |
73f05da to
92c0f6f
Compare
There was a problem hiding this comment.
Minor fix on-the-side to not use deprecated method any longer.
There was a problem hiding this comment.
New fix: On flush, we take the timestamp from the newValue if it exists.
There was a problem hiding this comment.
New fix: TupleForwarder should obey sendOldValues, too.
There was a problem hiding this comment.
New: slipped in the first commit of this PR... discovered while working on a follow up PR.
|
Rebased to resolve merge conflict. Also added some more fixes and tests. |
b79b478 to
21e5dee
Compare
|
LGTM. Please feel free to merge after green builds. |
|
retest this please |
…es-14-May * AK_REPO/trunk: (24 commits) KAFKA-7321: Add a Maximum Log Compaction Lag (KIP-354) (apache#6009) KAFKA-8335; Clean empty batches when sequence numbers are reused (apache#6715) KAFKA-6455: Session Aggregation should use window-end-time as record timestamp (apache#6645) KAFKA-6521: Use timestamped stores for KTables (apache#6667) [MINOR] Consolidate in-memory/rocksdb unit tests for window & session store (apache#6677) MINOR: Include StickyAssignor in system tests (apache#5223) KAFKA-7633: Allow Kafka Connect to access internal topics without cluster ACLs (apache#5918) MINOR: Align KTableAgg and KTableReduce (apache#6712) MINOR: Fix code section formatting in TROGDOR.md (apache#6720) MINOR: Remove unnecessary OptionParser#accepts method call from PreferredReplicaLeaderElectionCommand (apache#6710) KAFKA-8352 : Fix Connect System test failure 404 Not Found (apache#6713) KAFKA-8348: Fix KafkaStreams JavaDocs (apache#6707) MINOR: Add missing option for running vagrant-up.sh with AWS to vagrant/README.md KAFKA-8344; Fix vagrant-up.sh to work with AWS properly MINOR: docs typo in '--zookeeper myhost:2181--execute' MINOR: Remove header and key/value converter config value logging (apache#6660) KAFKA-8231: Expansion of ConnectClusterState interface (apache#6584) KAFKA-8324: Add close() method to RocksDBConfigSetter (apache#6697) KAFKA-6789; Handle retriable group errors in AdminClient API (apache#5578) KAFKA-8332: Refactor ImplicitLinkedHashSet to avoid losing ordering when converting to Scala ...
Reviewers: John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
The tests are failing because #6661 is not merged yet..
The goal of this PR is to swap out, KeyValueStore and WindowStore with the corresponding new TimestampedXxxStore in the DSL where appropriate. We don't leverage the timestamps yet in the
Processors, to keep the PR contained. Thus, no semantic change yet.