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-7277: Migrate Streams API to Duration instead of longMs times #5682

Merged
merged 21 commits into from
Oct 4, 2018

Conversation

nizhikov
Copy link
Collaborator

Public API changed according to KIP-358

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@nizhikov
Copy link
Collaborator Author

Hello, @vvcephei
Please, give me an advice: Should I also migrate all calls of the deprecated method in tests?

Copy link
Contributor

@vvcephei vvcephei left a comment

Choose a reason for hiding this comment

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

Hey @nizhikov ,

Thanks for the PR!

I've made a quick pass and left some comments that I think will apply to basically all of the methods, so I'll stop here.

At a high level, this is looking really good.

Please let me know if you have questions/objections to my feedback.

Thanks,
-John

throw new IllegalArgumentException(name + " shouldn't be null.", e);
} catch (final ArithmeticException e) {
throw new IllegalArgumentException(name + " can't be converted to milliseconds. " + d +
" is negative or too big", e);
Copy link
Contributor

Choose a reason for hiding this comment

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

This validation doesn't actually check that it's non-negative. I'm not sure that we want to, though.
Also, I'd use Objects.toString() on name just in case it's null.
Finally, it might be nice to wrap the substituted variables with brackets to make the result more readable.
All together, this would yield:

  • "[" + Objects.toString(name) + "] shouldn't be null.", e);
  • "[" + Objects.toString(name) + "] is too big to be converted to milliseconds: [" + d + "]", e);

What do you think about this?
(also below)

* @param d Duration to check
* @param name Name of params for an error message.
*/
public static void validateMillisecondDuration(final Duration d, final String name) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Please avoid single-letter variables: d -> duration
(also below)

@@ -63,7 +64,7 @@
@SuppressWarnings("unchecked")
public void init(final ProcessorContext context) {
this.context = context;
this.context.schedule(1000, PunctuationType.STREAM_TIME, timestamp -> {
this.context.schedule(Duration.ofMillis(1000), PunctuationType.STREAM_TIME, timestamp -> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Feel free to add a static import for ofMillis and STREAMS_TIME to shorten this line.

* @throws IllegalArgumentException if {@param timeout} is negative or too big
*/
public synchronized boolean close(final Duration timeout) throws IllegalArgumentException {
log.debug("Stopping Streams client with timeoutMillis = {} ms.", timeout.toMillis());
Copy link
Contributor

Choose a reason for hiding this comment

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

We should use validateMillisecondDuration here.

* the timestamp of the record from the primary stream.
*
* @param timeDifference join window interval
* @throws IllegalArgumentException if {@code timeDifference} is negative or too big
Copy link
Contributor

Choose a reason for hiding this comment

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

As I mentioned before, we don't throw IAE if timeDifference is negative.
(also below)

@@ -109,11 +110,27 @@ private JoinWindows(final long beforeMs,
* the timestamp of the record from the primary stream.
*
* @param timeDifferenceMs join window interval in milliseconds
* @throws IllegalArgumentException if {@code timeDifferenceMs} is negative
* @throws IllegalArgumentException if {@code timeDifferenceMs} is negative or too big
Copy link
Contributor

Choose a reason for hiding this comment

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

This method is actually guaranteed not to throw this exception. Since the parameter is a long of milliseconds, it's not possible to pass a value that is not expressible in milliseconds as a long.
(also applies elsewhere)

@nizhikov
Copy link
Collaborator Author

@vvcephei I refactor new methods. So they now call deprecated methods.
Please, review.

@vvcephei
Copy link
Contributor

I'll take a look tomorrow. Thanks @nizhikov .

@mjsax mjsax added the streams label Sep 26, 2018
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams;
Copy link
Member

Choose a reason for hiding this comment

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

This is a public package and thus would make this class public API -- don't think we want this (also not part of the KIP). Maybe, we should create a new package org.apache.kafka.streams.internals and move it there?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

moved


/**
*/
public final class ApiUtils {
Copy link
Member

Choose a reason for hiding this comment

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

I am wondering, what we gain by introducing this class?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Hello @mjsax

As discussed in KIP thread we want to wrap NPE and ArithmeticException into IllegalArgumentException

See @vvcephei suggestion: "I still feel that surfacing the ArithmeticException directly would not be a great experience, so I still advocate for wrapping it in an IllegalArgumentException that explains our upper bound for Duration is "max-long number of milliseconds"

* ApiUtils moved to internals package.
* JavaDoc fixes.
@nizhikov
Copy link
Collaborator Author

Hello @mjsax @vvcephei. I've fixed all your comments. Please, review.

Copy link
Contributor

@vvcephei vvcephei left a comment

Choose a reason for hiding this comment

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

Hi @nizhikov ,
I made another pass over this.

Mostly minor comments, with the exception of the one on streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java

Thanks again,
-John

* Typos fixed.
* Error messages improved.
* Wrong new method removed
* fetch, fetchAll added to WindowStore
@nizhikov
Copy link
Collaborator Author

Hello @vvcephei.
Thanks, for review.
I've fixed all your comments.

@vvcephei
Copy link
Contributor

Failure unrelated.

Retest this, please.

1 similar comment
@nizhikov
Copy link
Collaborator Author

Failure unrelated.

Retest this, please.

@nizhikov
Copy link
Collaborator Author

@vvcephei @mjsax I've checked failed tests locally and it all succeed.
Seems it some issue with test environment.
Please, give me an advice, what should I do?

dragon:~/src/kafka:[KAFKA-7277]$ scala -version
Scala code runner version 2.12.3 -- Copyright 2002-2017, LAMP/EPFL and Lightbend, Inc.
dragon:~/src/kafka:[KAFKA-7277]$ ./gradlew core:test --tests kafka.server.epoch.EpochDrivenReplicationProtocolAcceptanceTest.offsetsShouldNotGoBackwards


> Configure project :
Building project 'core' with Scala version 2.11.12
Building project 'streams-scala' with Scala version 2.11.12

> Task :core:test

kafka.server.epoch.EpochDrivenReplicationProtocolAcceptanceTest > offsetsShouldNotGoBackwards PASSED

BUILD SUCCESSFUL in 1m 15s
12 actionable tasks: 2 executed, 10 up-to-date
dragon:~/src/kafka:[KAFKA-7277]$ ./gradlew core:integrationTest --tests kafka.api.SaslSslAdminClientIntegrationTest.testMinimumRequestTimeouts

> Configure project :
Building project 'core' with Scala version 2.11.12
Building project 'streams-scala' with Scala version 2.11.12

> Task :core:integrationTest

kafka.api.SaslSslAdminClientIntegrationTest > testMinimumRequestTimeouts PASSED

BUILD SUCCESSFUL in 11s
10 actionable tasks: 2 executed, 8 up-to-date
dragon:~/src/kafka:[KAFKA-7277]$ ./gradlew core:test --tests kafka.admin.ConfigCommandTest.shouldExitWithNonZeroStatusOnArgError

> Configure project :
Building project 'core' with Scala version 2.11.12
Building project 'streams-scala' with Scala version 2.11.12

> Task :core:test

kafka.admin.ConfigCommandTest > shouldExitWithNonZeroStatusOnArgError PASSED

BUILD SUCCESSFUL in 2s
12 actionable tasks: 2 executed, 10 up-to-date

@nizhikov
Copy link
Collaborator Author

Failure unrelated.

Retest this, please.

1 similar comment
@nizhikov
Copy link
Collaborator Author

Failure unrelated.

Retest this, please.

@nizhikov
Copy link
Collaborator Author

@vvcephei @mjsax Finally, tests passed. Please, review.

Copy link
Contributor

@vvcephei vvcephei left a comment

Choose a reason for hiding this comment

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

Thanks for the update @nizhikov ,

I have just one more request (which applies multiple times); otherwise, it looks good to me!

@@ -179,6 +182,7 @@ public synchronized void put(final Bytes key, final byte[] value, final long win
}

@Override
@Deprecated
Copy link
Contributor

Choose a reason for hiding this comment

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

These don't need to be marked deprecated, thanks to the non-deprecated definition in WindowStore now.

This comment applies to all the implementations of WindowStore in this PR.

Copy link
Contributor

@vvcephei vvcephei left a comment

Choose a reason for hiding this comment

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

Looks good to me!

Thanks @nizhikov

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 1, 2018

@mjsax I don't want to waste your time :)
Can you share your IDE config options regarding code style?
In Apache Ignite we have done this in wiki

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 1, 2018

@mjsax Tests passed. What else should be done for merge?

Copy link
Member

@bbejeck bbejeck left a comment

Choose a reason for hiding this comment

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

Thanks @nizhikov for the patch. I left some general comments but overall LGTM.

/**
* Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the
* threads to join.
* A {@code timeout} of 0 means to wait forever.
Copy link
Member

Choose a reason for hiding this comment

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

Overall I'm ok with the semantics.

But my first instinct of a timeout of 0 implies shutdown immediately with no wait and blocking forever takes a value of Long.MAX_VALUE.

In short, I'm +1 as well on revising the behavior.

@@ -361,7 +364,7 @@ public boolean conditionMet() {
final int index = metadata.hostInfo().port();
final KafkaStreams streamsWithKey = streamRunnables[index].getStream();
final ReadOnlyWindowStore<String, Long> store = streamsWithKey.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
return store != null && store.fetch(key, from, to) != null;
return store != null && store.fetch(key, ofEpochMilli(from), ofMillis(to - from)) != null;
Copy link
Member

Choose a reason for hiding this comment

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

For me, I have a slight preference for the from - to approach, but that's due more to personal preference.
I agree with @vvcephei that either way it's the same semantics just expressed differently.

So overall I'd prefer Instant, Instant, but if others want Instant, Duration I'm good with that as well.

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 2, 2018

I've resolved merge conflicts.

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 2, 2018

Failure unrelated.

Retest this, please.

* @throws NullPointerException If {@code null} is used for any key.
* @throws IllegalArgumentException if duration is negative or can't be represented as {@code long milliseconds}
*/
KeyValueIterator<Windowed<K>, V> fetch(K from, K to, Instant fromTime, Duration duration)
Copy link
Member

Choose a reason for hiding this comment

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

I just talked to @guozhangwang @bbejeck and @vvcephei -- it seems, we are in favor to go with Instant/Instant types here and also for fetchAll(). 2.1 branch is not cut, and thus we can still get this into 2.1 -- can you update the PR accordingly? (Sorry for the long delay, but we were quite busy with many other deadline related things...)

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

@mjsax

I've updated (KIP-358)[https://cwiki.apache.org/confluence/display/KAFKA/KIP-358%3A+Migrate+Streams+API+to+Duration+instead+of+long+ms+times] and PR according to your proposal.

Please, review.

@@ -361,7 +364,7 @@ public boolean conditionMet() {
final int index = metadata.hostInfo().port();
final KafkaStreams streamsWithKey = streamRunnables[index].getStream();
final ReadOnlyWindowStore<String, Long> store = streamsWithKey.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
return store != null && store.fetch(key, from, to) != null;
return store != null && store.fetch(key, ofEpochMilli(from), ofMillis(to - from)) != null;
Copy link
Member

Choose a reason for hiding this comment

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

Just synced up with the others -- seems we are in favor to switch to Instant/Instant and try to get it into 2.1 (see my other comment).

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 4, 2018

@mjsax @bbejeck
I migrated fetch methods to Instant, Instant.
Tests passed.
Please, review.

@vvcephei
Copy link
Contributor

vvcephei commented Oct 4, 2018

Hey @nizhikov , I apologise that I didn't think of this earlier, but can you provide default implementations of the new fetch methods in the WindowStore interface? The default implementations would just defer to the long/long methods.

This would mean that any implementations would just automatically work with the new APIs. In fact, you could then pull the implementations out of all our window store implementations.

Does that make sense?

(I'm tempted not to bring it up at this late stage, but I think it would be very nice for store implementers not to have to add the trivial delegate methods.)

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 4, 2018

@vvcephei Do you mean we need to add default implementations into ReadOnlyWindowStore?

@vvcephei
Copy link
Contributor

vvcephei commented Oct 4, 2018

No, I mean in WindowStore (the Instant/Instant methods are available there, since WindowStore extends ReadOnlyWindowStore), and the long/long methods are not deprecated in this context.

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 4, 2018

@vvcephei

But we have classes that only extends ReadOnlyWindowStore. CompositeReadOnlyWindowStore, for example. So if we want to reduce code duplication we have to add default implementations to ReadOnlyWindowStore. Isn't it?

@mjsax
Copy link
Member

mjsax commented Oct 4, 2018

@nizhikov @vvcephei

Because we deprecate ReadOnlyWindowStore#fetch(K, long, long) I don't think it makes sense to add default implementation there, that "map" from fetch(K, Instant, Instant) to #fetch(K, long, long)

However, I am not even sure how much we gain adding any default implementation. I am not against it, however, only people who implement their own WindowStore would be affected and those are very few... Might not be worth it.

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 4, 2018

@mjsax @vvcephei So maybe just merge it? :)

Copy link
Member

@bbejeck bbejeck left a comment

Choose a reason for hiding this comment

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

Thanks for the quick change @nizhikov. The changes LGTM

@vvcephei
Copy link
Contributor

vvcephei commented Oct 4, 2018

Sounds good to me.

@mjsax
Copy link
Member

mjsax commented Oct 4, 2018

Cool. I'll wait until Jenkins passes (and go over the changes -- my initial "skip over review" looked good) and than merge, if @guozhangwang does not raise any concern.

@vvcephei
Copy link
Contributor

vvcephei commented Oct 4, 2018

Many thanks for this feature (and your patience) @nizhikov !

Copy link
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

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

Thanks for the contribution @nizhikov !

@nizhikov
Copy link
Collaborator Author

nizhikov commented Oct 4, 2018

@mjsax Tests passed.

@mjsax mjsax merged commit ca641b3 into apache:trunk Oct 4, 2018
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants