KAFKA-8615: Change to track partition time breaks TimestampExtractor#7054
KAFKA-8615: Change to track partition time breaks TimestampExtractor#7054guozhangwang merged 13 commits intoapache:trunkfrom
Conversation
|
Also, I think we need to add back the notion of partition time. Currently, we pass in |
bbejeck
left a comment
There was a problem hiding this comment.
Thanks for the patch @ableegoldman, just one minor comment otherwise lgtm modulo comments from @mjsax.
|
retest this please |
mjsax
left a comment
There was a problem hiding this comment.
Some follow up comments, based on our in-person discussion today.
| final long timestamp; | ||
| try { | ||
| timestamp = timestampExtractor.extract(deserialized, timestamp()); | ||
| timestamp = timestampExtractor.extract(deserialized, partitionTime); |
There was a problem hiding this comment.
Can we also update method timestamp() to headRecordTimestamp() to be more explicit what it returns? It's orthogonal to the actual fix, but might be a good improvement.
There was a problem hiding this comment.
Similar, can we piggy-back some cleanup to PartitionGroup ?
- rename PartitionGroup#timestamp() to PartitionGroup#streamTime()
(also update the JavaDocs, that seems to be wrong)
- in `clear()` reset `streamTime` to UNKNOWN ?
- in `nextRecord()`: do we need to check if `queue != null` and do we need to check if `record != null` (seem it's ensure that both can never be `null` ?)
There was a problem hiding this comment.
Ack to all...except the last point. We do check both for null..?
There was a problem hiding this comment.
Seems, we check both for null atm:
final RecordQueue queue = nonEmptyQueuesByTime.poll();
info.queue = queue;
if (queue != null) {
// get the first record from this queue.
record = queue.poll();
if (record != null) {
--totalBuffered;
But I think they cannot be null, could they?
There was a problem hiding this comment.
Sorry, misunderstood your question. Yes, either one could potentially be null if we don't yet have new records to process?
There was a problem hiding this comment.
Good point. final RecordQueue queue = nonEmptyQueuesByTime.poll(); could return null. However, I am wondering if record = queue.poll(); could return null, because it's called nonEmptyQueuesByTime -- hence, queue should never be empty?
There was a problem hiding this comment.
I think I agree the second null check should never happen.
| @@ -41,8 +41,8 @@ | |||
| * | |||
| * PartitionGroup also maintains a stream-time for the group as a whole. | |||
There was a problem hiding this comment.
Maybe we should change "stream-time" to "task-time" ? @vvcephei suggested it (and I like it) in an in-person discussion? If we agree, also the corresponding variable and method names should be updated. Thoughts?
Should it be "stream-time" or "stream time" ?
There was a problem hiding this comment.
I support "task-time" -- would be good to distinguish from "stream time" should we ever want/need a "global" stream time. That said, we use stream time all over including things like PunctuationType. Will it be confusing to have maybePuncuateStreamTime() punctuate on something called task-time? Though we already refer to it separately as "partition group timestamp", "stream partition time", AND "stream time" in that method..
Regarding "stream-time" vs "stream time" -- we use the hyphen when referring to types of time semantics (eg event-time) so I'd favor "stream time" to maintain a distinction between "semantics types" and "time definitions" -- WDYT?
There was a problem hiding this comment.
On the other hand, maybe phrases like "lowest task time" might be ambiguous (what does "lowest task" mean and why do we care about its time?) so I'm good with task-time.
| * The PartitionGroup's stream-time is also the stream-time of its task and is used as the | ||
| * stream-time for any computations that require it. | ||
| * Note however that any computation that depends on stream time tracks it on a per-operator basis to obtain an | ||
| * accurate view of the local stream time as seen by that node. |
There was a problem hiding this comment.
node -> processor? Maybe we could call this "processor time" ?
There was a problem hiding this comment.
How about just local stream time -> local time to avoid introducing too many types of time? Or do you feel "processor time" is clear enough (possibly more so than "local time" -- WDYT?)
There was a problem hiding this comment.
In light of renaming "stream time" -> "task time" I think it does make sense to call this "processor time" and establish a clear naming hierarchy
| final long timestamp; | ||
| try { | ||
| timestamp = timestampExtractor.extract(deserialized, timestamp()); | ||
| timestamp = timestampExtractor.extract(deserialized, partitionTime); |
There was a problem hiding this comment.
Seems, we check both for null atm:
final RecordQueue queue = nonEmptyQueuesByTime.poll();
info.queue = queue;
if (queue != null) {
// get the first record from this queue.
record = queue.poll();
if (record != null) {
--totalBuffered;
But I think they cannot be null, could they?
|
retest this please |
|
java 11 scala 2.13 timed out, java 8 failure unrelated retest this please |
|
java 11 scala 2.12 failed, java 8 failed retest this please |
|
Java11/2.12: (https://issues.apache.org/jira/browse/KAFKA-8672) Java11/2.13 (https://issues.apache.org/jira/browse/KAFKA-8555) Java8: env error Retest this please. |
|
Java8 failed again with env error Retest this please. |
|
Java8 and 11.12 passed, Java11.13 failed with known |
| * Return the stream-time of this partition group defined as the largest timestamp seen across all partitions | ||
| */ | ||
| public long timestamp() { | ||
| public long streamTime() { |
There was a problem hiding this comment.
Should we rename this this taskTime()? (Just a thought).
There was a problem hiding this comment.
I'm in favor of that in theory -- but, then do we also rename maybePuncuateStreamTime ? Do we also deprecate PunctuationType.STREAM_TIME in favor of PunctuationType.TASK_TIME? Task time does seem more appropriate but I'm hesitant to mix terminology ...
There was a problem hiding this comment.
Let's just keep it as streamTime for now.
guozhangwang
left a comment
There was a problem hiding this comment.
LGTM. @ableegoldman I think we can remove the second null check but I'm okay keeping it as well just to be safe.
| * Return the stream-time of this partition group defined as the largest timestamp seen across all partitions | ||
| */ | ||
| public long timestamp() { | ||
| public long streamTime() { |
There was a problem hiding this comment.
Let's just keep it as streamTime for now.
| final long timestamp; | ||
| try { | ||
| timestamp = timestampExtractor.extract(deserialized, timestamp()); | ||
| timestamp = timestampExtractor.extract(deserialized, partitionTime); |
There was a problem hiding this comment.
I think I agree the second null check should never happen.
…7054) The timestamp extractor takes a previousTimestamp parameter which should be the partition time. This PR adds back in partition time tracking for the extractor, and renames previousTimestamp --> partitionTime Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Matthias J. Sax <mjsax@apache.org>
|
Cherry-picked to 2.3 as well. |
|
Thanks @guozhangwang! I think it should actually be cherry-picked all the way back to 2.1 I don't think there should be conflicts but if you need a separate PR for the earlier branches, let me know |
…7054) The timestamp extractor takes a previousTimestamp parameter which should be the partition time. This PR adds back in partition time tracking for the extractor, and renames previousTimestamp --> partitionTime Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Matthias J. Sax <mjsax@apache.org>
…7054) The timestamp extractor takes a previousTimestamp parameter which should be the partition time. This PR adds back in partition time tracking for the extractor, and renames previousTimestamp --> partitionTime Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Matthias J. Sax <mjsax@apache.org>
|
Cherry-picked to 2.2 and 2.1. |
* apache-github/2.3: MINOR: Update documentation for enabling optimizations (apache#7099) MINOR: Remove stale streams producer retry default docs. (apache#6844) KAFKA-8635; Skip client poll in Sender loop when no request is sent (apache#7085) KAFKA-8615: Change to track partition time breaks TimestampExtractor (apache#7054) KAFKA-8670; Fix exception for kafka-topics.sh --describe without --topic mentioned (apache#7094) KAFKA-8602: Separate PR for 2.3 branch (apache#7092) KAFKA-8530; Check for topic authorization errors in OffsetFetch response (apache#6928) KAFKA-8662; Fix producer metadata error handling and consumer manual assignment (apache#7086) KAFKA-8637: WriteBatch objects leak off-heap memory (apache#7050) KAFKA-8620: fix NPE due to race condition during shutdown while rebalancing (apache#7021) HOT FIX: close RocksDB objects in correct order (apache#7076) KAFKA-7157: Fix handling of nulls in TimestampConverter (apache#7070) KAFKA-6605: Fix NPE in Flatten when optional Struct is null (apache#5705) Fixes apache#8198 KStreams testing docs use non-existent method pipe (apache#6678) KAFKA-5998: fix checkpointableOffsets handling (apache#7030) KAFKA-8653; Default rebalance timeout to session timeout for JoinGroup v0 (apache#7072) KAFKA-8591; WorkerConfigTransformer NPE on connector configuration reloading (apache#6991) MINOR: add upgrade text (apache#7013) Bump version to 2.3.1-SNAPSHOT
… breaks TimestampExtractor (apache#7054) TICKET = KAFKA-8615 LI_DESCRIPTION = EXIT_CRITERIA = HASH [3d7b989] ORIGINAL_DESCRIPTION = The timestamp extractor takes a previousTimestamp parameter which should be the partition time. This PR adds back in partition time tracking for the extractor, and renames previousTimestamp --> partitionTime Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Matthias J. Sax <mjsax@apache.org> (cherry picked from commit 3d7b989)
The timestamp extractor takes a
previousTimestampparameter which should be the partition time. This PR adds back in partition time tracking for the extractor, and renamespreviousTimestamp-->partitionTimeShould be cherry-picked back to 2.1