KAFKA-7245 (Deprecate WindowStore#put(key, value)) :- The method in t…#7105
KAFKA-7245 (Deprecate WindowStore#put(key, value)) :- The method in t…#7105mjsax merged 15 commits intoapache:trunkfrom
Conversation
…he WindowStore#put(key, value) has no timestamp as parameter, so it causes inconsistency to identify to which window does the key belong.
|
Thanks for the PR @omanges -- seems the code does not compile... Please update your PR to make it compile so we can review it. |
|
Copied from the Jira ticket:
The build does not run any tests but fails because it does not compile (note, that compiler warning will fail the build):
Do you refer to a local run of the tests? If yes, the question is why exactly did the tests fail? |
|
warning: [deprecation] put(K,V) in WindowStore has been deprecated Similar types of warnings are generated. Due to this, the local build is also not able to compile. So does this means we also need to change test cases as well which are using the method which we have tagged as deprecated? |
|
Yes, you need to add |
…he WindowStore#put(key, value) has no timestamp as parameter, so it causes inconsistency to identify to which window does the key belong.
|
I am able to build successfully on local. |
|
Checkstyle failed. Try to execute |
…he WindowStore#put(key, value) has no timestamp as parameter, so it causes inconsistency to identify to which window does the key belong.
| * to identify the window frame to which the key belongs. | ||
| * Use WindowStore#put(key, value, timestamp) instead. | ||
| * | ||
| */ |
There was a problem hiding this comment.
We don't need JavaDocs here as we inherit JavaDocs by default automatically. (Similar for all other classes that extend the top level interface -- it's sufficient to have JavaDocs at the top level interface.)
| * if the serialized bytes are also null it is interpreted as deletes | ||
| * @throws NullPointerException if the given key is {@code null} | ||
| * | ||
| * @deprecated as timestamp is not provided for the (key, value) pair, this causes inconsistency |
| * | ||
| * @deprecated as timestamp is not provided for the (key, value) pair, this causes inconsistency | ||
| * to identify the window frame to which the key belongs. | ||
| * Use WindowStore#put(key, value, timestamp) instead. |
There was a problem hiding this comment.
Nit: Use {@link #put(Object, Object, long)} instead.
| runGenericBenchmark(streams, "Streams Stateful Performance [records/latency/rec-sec/MB-sec joined]: ", latch); | ||
| } | ||
|
|
||
| @SuppressWarnings("deprecation") |
There was a problem hiding this comment.
I think for SimpleBenchmark we should update the code instead of suppressing the warning.
There was a problem hiding this comment.
So the method call store.put(key, value), can be replaced by store.put(key, value, timestamp) as we are able to retrieve timestamp from the context. @mjsax what do you think?
There was a problem hiding this comment.
I have updated the code in the latest commit please have a look.
| } | ||
|
|
||
| @Test | ||
| @SuppressWarnings("deprecation") |
There was a problem hiding this comment.
This this and the other tests in this class: I am not sure what the best approach is, and if we should rather update the code -- or maybe even duplicate the code?
Similar for some other test classes.
\cc @guozhangwang @bbejeck @vvcephei @ableegoldman @cadonna @abbccdda WDYT?
There was a problem hiding this comment.
Personally I think we should migrate all these tests to use the non-deprecated put then add a test or two just verifying put(key) and put(key, time) have the same results. But IMO all the tests which aren't explicitly testing the deprecated method should be using the "correct" one
There was a problem hiding this comment.
I'd agree with @ableegoldman : for test coverage of functionality, we should touch on non-deprecated code only; and only having a few deprecated test that tests the deprecated methods acts the same behavior as their delegated method.
There was a problem hiding this comment.
Thanks! That sounds reasonable to me!
There was a problem hiding this comment.
Hi, @mjsax the timestamp is not present in the local context of the test cases. So in order to replace the deprecated method with the new method, we need some timestamp. So can we use the 'DEFAULT_TIMESTAMP' which defined as a global in the class CachingWindowStoreTest.
There was a problem hiding this comment.
For shouldNotReturnDuplicatesInRanges it seems best to use processorContext.timestamp() -- processorContext is passed in init() do you just need to add a member variable to the Transformer to store is so you can use it in transform()
There was a problem hiding this comment.
Hi, @mjsax I have made changes for the shouldNotReturnDuplicatesInRanges. For the remaining usage i.e for the cachingStore do we need to update to make use of the method with timestamp?
There was a problem hiding this comment.
Hi, @mjsax it seems that an exception is thrown if a call is made to access the timestamp in the init.
Exception:-
org.apache.kafka.streams.state.internals.CachingWindowStoreTest > shouldNotReturnDuplicatesInRanges FAILED
org.apache.kafka.streams.errors.StreamsException: failed to initialize processor KSTREAM-TRANSFORM-0000000001
at org.apache.kafka.streams.processor.internals.ProcessorNode.init(ProcessorNode.java:96)
at org.apache.kafka.streams.processor.internals.StreamTask.initTopology(StreamTask.java:526)
at org.apache.kafka.streams.processor.internals.StreamTask.initializeTopology(StreamTask.java:253)
at org.apache.kafka.streams.TopologyTestDriver.(TopologyTestDriver.java:377)
at org.apache.kafka.streams.TopologyTestDriver.(TopologyTestDriver.java:240)
at org.apache.kafka.streams.state.internals.CachingWindowStoreTest.shouldNotReturnDuplicatesInRanges(CachingWindowStoreTest.java:180)
Caused by:
java.lang.IllegalStateException: This should not happen as timestamp() should only be called while a record is processed
at org.apache.kafka.streams.processor.internals.AbstractProcessorContext.timestamp(AbstractProcessorContext.java:158)
at org.apache.kafka.streams.state.internals.CachingWindowStoreTest$1.init(CachingWindowStoreTest.java:134)
at org.apache.kafka.streams.kstream.internals.TransformerSupplierAdapter$1.init(TransformerSupplierAdapter.java:42)
at org.apache.kafka.streams.kstream.internals.KStreamFlatTransform$KStreamFlatTransformProcessor.init(KStreamFlatTransform.java:51)
at org.apache.kafka.streams.processor.internals.ProcessorNode.init(ProcessorNode.java:92)
There was a problem hiding this comment.
That is expected. You need to add a member variable ProcessorContext context to the class and assign this.context = context in init().
In transform() you can use context.timestamp() to get the current timestamp.
| }); | ||
| } | ||
|
|
||
|
|
…he WindowStore#put(key, value) has no timestamp as parameter, so it causes inconsistency to identify to which window does the key belong.
…he WindowStore#put(key, value) has no timestamp as parameter, so it causes inconsistency to identify to which window does the key belong.
|
Is Jenkins server down? I am not able to see the cause of the build fail. |
|
Build results get removed after a few days. Probably a flaky test, but try running locally just in case. Retest this, please |
|
Can someone help me I am not able to understand the reason for the failure of the test cases? |
|
How can I trigger tests again? |
|
You can retrigger them by commenting: retest this, please |
|
Hi please retest this. |
|
The test is not failing because of the changes I am unable to understand, what’s the problem. |
|
Yeah, unfortunately there are a number of flaky tests across kafka projects. They can fail for environmental reasons not related to your PR. If you see a failure you believe is unrelated to your changes, please submit a JIRA for the flaky test (or comment on an existing one to note there was an additional failure). Generally we just comment on the PR saying which builds passed and which test failed on the ones that didn't, then retrigger the tests. |
|
@omanges -- I would recommend to run the test locally first and make sure they pass. If they pass locally and there are test failures on Jenkins, it might be due to test flakiness. Did you run the tests locally? Also to trigger a Jenkins retest, you need to use the exact phrase |
…he WindowStore#put(key, value) has no timestamp as parameter, so it causes inconsistency to identify to which window does the key belong.
|
@mjsax I think the first one failed due to some flaky test. |
|
So are there any further changes needed for this PR? |
Implements KIP-474. Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Matthias J. Sax <matthias@confluent.io>
|
Merged to Thanks for the KIP and PR @omanges! |
|
Thanks. |
…t-for-generated-requests * apache-github/trunk: KAFKA-8932; Add tag for CreateTopicsResponse.TopicConfigErrorCode (KIP-525) (apache#7464) KAFKA-8944: Fixed KTable compiler warning. (apache#7393) KAFKA-8964: Rename tag client-id for thread-level metrics and below (apache#7429) MINOR: remove unused imports in Streams system tests (apache#7468) KAFKA-7190; Retain producer state until transactionalIdExpiration time passes (apache#7388) KAFKA-8983; AdminClient deleteRecords should not fail all partitions unnecessarily (apache#7449) MINOR: Modified Exception handling for KIP-470 (apache#7461) KAFKA-7245: Deprecate WindowStore#put(key, value) (apache#7105) KAFKA-8179: Part 7, cooperative rebalancing in Streams (apache#7386) KAFKA-8985; Add flexible version support to inter-broker APIs (apache#7453) MINOR: Bump version to 2.5.0-SNAPSHOT (apache#7455)
WindowStore#put(key, value) has no timestamp as a parameter, so it causes inconsistency to identify to which window does the key belong.
The stream module of Kafka has a window state store that stores the aggregated values for a key in a given time frame. The window store is implemented as an interface, this interface has a strange method named put(key, value), this method has does not have a timestamp as a parameter which is important to determine that to which window frame does the key belongs. In this method, the current record timestamp is used for determining the window frame(as specified in the description of the method), this constraint makes WindowStore error-prone. It is also specified in the method description that method with a timestamp parameter should be used which already present in the interface which expects key, value, and start timestamp as well of the window to which the key belongs. Therefore by deprecating (and finally removing) the method put(key, value), we can prevent inconsistency.
Committer Checklist (excluded from commit message)