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-5233 KIP-138: Change punctuate semantics #3055

Closed
wants to merge 18 commits into from

Conversation

mihbor
Copy link
Contributor

@mihbor mihbor commented May 15, 2017

Implementation for KIP-138: Change punctuate semantics

@mihbor mihbor changed the title Kip 138 KAFKA-5233 KIP-138 May 15, 2017
@asfbot
Copy link

asfbot commented May 15, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/3914/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented May 15, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/3925/
Test FAILed (JDK 7 and Scala 2.11).

@asfbot
Copy link

asfbot commented May 15, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/3927/
Test FAILed (JDK 7 and Scala 2.11).

@asfbot
Copy link

asfbot commented May 15, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/3917/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented May 15, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/3928/
Test PASSed (JDK 7 and Scala 2.11).

@enothereska
Copy link
Contributor

retest this please

@asfbot
Copy link

asfbot commented May 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/3984/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link

asfbot commented May 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/3987/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link

asfbot commented May 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/3975/
Test PASSed (JDK 8 and Scala 2.12).

if (processorContext.currentNode() != null) {
throw new IllegalStateException(String.format("%s Current node is not null", logPrefix));
}

updateProcessorContext(new StampedRecord(DUMMY_RECORD, timestamp), node);

log.trace("{} Punctuating processor {} with timestamp {}", logPrefix, node.name(), timestamp);
log.trace("{} Punctuating processor {} with timestamp {} {}", logPrefix, node.name(), timestamp, type);
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: "with timestamp {} and type {}"

@enothereska
Copy link
Contributor

Did a first pass over interfaces and depreciated functions, looks good. Haven't covered tests yet. FYI. @mihbor any chance you could rebase to latest trunk? Thanks.

@enothereska
Copy link
Contributor

@mihbor a key thing that seems to be missing seems to be that if no events arrive, the system timer will never trigger. So we have again the issue that in absence of events, punctuate is never called. I could be missing something. Is the way around this to call maybePunctuate in the runLoop method in StreamThread.java right before calling maybeCommit? That way we can check the time after each poll. cc @mjsax @guozhangwang on this as well. This, and associated tests around system time, seem to be key things missing IMO. Thanks.

Copy link
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

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

I agree with @enothereska about missing improvements on when to punctuate.

@@ -82,6 +83,7 @@
R transform(final K key, final V value);

/**
* <b>Deprecated as of 0.11.0.0</b> - <i>Please use {@link Punctuator)} functional interface instead.</i>
Copy link
Member

Choose a reason for hiding this comment

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

We should add @Deprecated Java annotation, and also use JavaDoc tag @Deprecated

@@ -44,11 +44,13 @@
void process(K key, V value);

/**
* <b>Deprecated as of 0.11.0.0</b> - <i>Please use {@link Punctuator)} functional interface instead.</i>
Copy link
Member

Choose a reason for hiding this comment

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

Please use JavaDoc tag @Deprecated instead

* schedule a periodic call called a punctuation to {@link Punctuator#punctuate(long)}.
*
* @param interval the time interval between punctuations
* @param type one of:<ul>>
Copy link
Member

Choose a reason for hiding this comment

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

nit: markup, two >>

I would also prefer to have this explanation in the body and not in the parameter list.

* in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use.
* <b>NOTE:</b> Only advanced if messages arrive</li>
* <li>{@link PunctuationType#SYSTEM_TIME} - uses the system time (aka wall-clock time),
* which is advanced at the polling interval ({@link org.apache.kafka.streams.StreamsConfig#POLL_MS_CONFIG})
Copy link
Member

Choose a reason for hiding this comment

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

advanced at the polling interval: not sure if this is true.

@Override
public <K, V> void forward(K key, V value, int childIndex) {
throw new UnsupportedOperationException();
throw new UnsupportedOperationException("this should not happen: forward() not supported in global processor context.");
Copy link
Member

Choose a reason for hiding this comment

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

Thanks for the cleanup! Appreciate it!

task.schedule(interval);
schedule(interval, PunctuationType.STREAM_TIME, new Punctuator() {
@Override
public void punctuate(long timestamp) {
Copy link
Member

Choose a reason for hiding this comment

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

nit: add final

@@ -122,8 +125,19 @@ public void commit() {
}

@Override
public Cancellable schedule(long interval, PunctuationType type, Punctuator callback) {
Copy link
Member

Choose a reason for hiding this comment

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

nit: add final to all parameters

@@ -422,12 +446,14 @@ int numBuffered() {
boolean maybePunctuate() {
final long timestamp = partitionGroup.timestamp();

boolean punctuated = systemTimePunctuationQueue.mayPunctuate(System.currentTimeMillis(), PunctuationType.SYSTEM_TIME, this);
Copy link
Member

Choose a reason for hiding this comment

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

we should not call System.currentTimeMillis() directly, but use the tasks Time object -- otherwise, we cannot test anything.

Copy link
Member

Choose a reason for hiding this comment

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

With regard to high level comment, we should on do this call here anyway.

@mihbor
Copy link
Contributor Author

mihbor commented May 27, 2017

Oh, I see your point about when to punctuate, thanks.
Admittedly, I was rushing to get the PR submitted before the cut-off. Now, since that ship has sailed I'll have a more thorough look and work on test coverage as well.

@asfbot
Copy link

asfbot commented May 27, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4485/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link

asfbot commented May 27, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4471/
Test PASSed (JDK 8 and Scala 2.12).

@mihbor
Copy link
Contributor Author

mihbor commented May 27, 2017

Not sure about metrics. Should there be one punctuate time sensor or one for stream time and a separate one for system time punctuations?

@mihbor
Copy link
Contributor Author

mihbor commented May 27, 2017

My naive implementation of cancellable was quite badly broken as it wasn't pointing at the latest PunctuationSchedule object, but always only the first one. About to commit something more involved but functional.

@asfbot
Copy link

asfbot commented May 27, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4488/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link

asfbot commented May 27, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4474/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented May 27, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4490/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link

asfbot commented May 27, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4476/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented May 28, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4509/
Test PASSed (JDK 7 and Scala 2.11).

@asfbot
Copy link

asfbot commented May 28, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4494/
Test PASSed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented Jun 3, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/4856/
Test PASSed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented Jun 3, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/4872/
Test PASSed (JDK 7 and Scala 2.11).

@enothereska
Copy link
Contributor

Thanks, changes LGTM. Calling for second reviews @dguy @bbejeck

Copy link
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

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

Some nits. Overall LGTM.

@@ -28,8 +30,8 @@
* This is a stateful record-by-record operation, i.e, {@link #transform(Object, Object)} is invoked individually for
* each record of a stream and can access and modify a state that is available beyond a single call of
* {@link #transform(Object, Object)} (cf. {@link KeyValueMapper} for stateless record transformation).
* Additionally, the interface can be called in regular intervals based on the processing progress
* (cf. {@link #punctuate(long)}.
* Additionally, this Transformer can {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule}
Copy link
Member

Choose a reason for hiding this comment

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

Nit: {@code Transformer}

@@ -28,8 +30,8 @@
* This is a stateful record-by-record operation, i.e, {@link #transform(Object)} is invoked individually for each
* record of a stream and can access and modify a state that is available beyond a single call of
* {@link #transform(Object)} (cf. {@link ValueMapper} for stateless value transformation).
* Additionally, the interface can be called in regular intervals based on the processing progress
* (cf. {@link #punctuate(long)}.
* Additionally, this ValueTransformer can {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule}
Copy link
Member

Choose a reason for hiding this comment

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

Nit: {@code ValueTransformer}

@@ -82,6 +84,7 @@
R transform(final K key, final V value);

/**
* @deprecated As of 0.11.1.0 please use {@link Punctuator} functional interface instead.
Copy link
Member

Choose a reason for hiding this comment

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

Nit: remove As of 0.11.1.0 -- we also don't know yet what the next version number will be. If it's not 0.11.1.0 it would be wrong and I am pretty sure we will miss to update it.

@@ -82,6 +84,7 @@
VR transform(final V value);

/**
* @deprecated As of 0.11.1.0 please use {@link Punctuator} functional interface instead.
Copy link
Member

Choose a reason for hiding this comment

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

As above

@@ -44,11 +45,13 @@
void process(K key, V value);

/**
* @deprecated As of 0.11.1.0 please use {@link Punctuator} functional interface instead.
Copy link
Member

Choose a reason for hiding this comment

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

As above.

Cancellable schedule(long interval, PunctuationType type, Punctuator callback);

/**
* @deprecated As of 0.11.1.0 please use {@link #schedule(long, PunctuationType, Punctuator)} instead.
Copy link
Member

Choose a reason for hiding this comment

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

As above

@@ -88,11 +88,35 @@

/**
* Schedules a periodic operation for processors. A processor may call this method during
* {@link Processor#init(ProcessorContext) initialization} or processing
* {@link Processor#process(Object, Object)} to
* schedule a periodic call called a punctuation to {@link Punctuator#punctuate(long)}.
Copy link
Member

Choose a reason for hiding this comment

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

nit: a periodic call&mdash;called a punctuation&mdash;to

package org.apache.kafka.streams.processor;

/**
* A functional interface used as an argument to {@link ProcessorContext#schedule(long, PunctuationType, Punctuator)}
Copy link
Member

Choose a reason for hiding this comment

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

Super-nit: add . at the end

@mihbor
Copy link
Contributor Author

mihbor commented Jun 23, 2017

Sure, I'll change it. I'm also not a fan of premature optimisation and I think the impact would be negligible, but I didn't want to depart from the current practice for handling delegates in that class.

@mihbor
Copy link
Contributor Author

mihbor commented Jun 24, 2017

rebased

@asfgit
Copy link

asfgit commented Jun 24, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5688/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

asfgit commented Jun 24, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5674/
Test PASSed (JDK 8 and Scala 2.12).

Copy link
Contributor

@dguy dguy left a comment

Choose a reason for hiding this comment

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

LGTM

@dguy
Copy link
Contributor

dguy commented Jun 28, 2017

Thanks @mihbor for your efforts on this KIP! Merged to trunk

@asfgit asfgit closed this in efb060c Jun 28, 2017
@peoplemerge
Copy link

peoplemerge commented Aug 30, 2017

Thanks for this work, @mihbor

I added trunk dep to my code base, I found the interface change self-describing, even near the end of a 10 hour day coding. My code was:
this.context.schedule(150l);

and became:
this.context.schedule(150l, PunctuationType.SYSTEM_TIME, this);

It was with great relief I saw all of our tests magically flip from failure to pass. I requested a hotfix to KIP-138 for 0.11.0.0.

How's that for validation?

@mihbor
Copy link
Contributor Author

mihbor commented Aug 31, 2017

Thanks for your kind comment :-)

peoplemerge pushed a commit to peoplemerge/kafka that referenced this pull request Sep 13, 2017
Cherry-picker: Dave Thomas <david.thomas@ticketmaster.com>
Add revisionId to gradle-props.

KAFKA-5233; KIP-138: Change punctuate semantics

Implementation for KIP-138: Change punctuate semantics

Author: Michal Borowiecki <michal.borowiecki@openbet.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bbejeck@gmail.com>, Eno Thereska <eno.thereska@gmail.com>, Damian Guy <damian.guy@gmail.com>

Closes apache#3055 from mihbor/KIP-138
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants