Thanks +1 (binding)
On Sun, 16 Sep 2018 at 19:37 Nikolay Izhikov wrote:
> Dear commiters,
>
> I got two binding +1 in [VOTE] thread for this KIP [1].
> I still need one more.
>
> Please, take a look at KIP.
>
> [1]
> https://lists.apache.org/thread.html/e976352e7e42d459091ee66ac790b6a0de7064eac0
Dear commiters,
I got two binding +1 in [VOTE] thread for this KIP [1].
I still need one more.
Please, take a look at KIP.
[1]
https://lists.apache.org/thread.html/e976352e7e42d459091ee66ac790b6a0de7064eac0c57760d50c983b@%3Cdev.kafka.apache.org%3E
В Чт, 13/09/2018 в 19:33 +0300, Nikolay Izhik
Fixed.
Thanks, for help!
Please, take a look and vote.
В Чт, 13/09/2018 в 08:40 -0700, Matthias J. Sax пишет:
> No need to start a new voting thread :)
>
> For the KIP update, I think it should be:
>
> > ReadOnlyWindowStore {
> > //Deprecated methods.
> > WindowStoreIterator fetch(K k
No need to start a new voting thread :)
For the KIP update, I think it should be:
> ReadOnlyWindowStore {
> //Deprecated methods.
> WindowStoreIterator fetch(K key, long timeFrom, long timeTo);
> KeyValueIterator, V> fetch(K from, K to, long timeFrom, long
> timeTo);
> KeyValueIt
Hello, Matthias.
> I like the KIP as-is. Feel free to start a VOTE thread.
I'm already started one [1].
Can you vote in it or I should create a new one?
I've updated KIP.
This has been changed:
ReadOnlyWindowStore {
//Deprecated methods.
WindowStoreIterator fetch(K key, long timeFrom,
Great!
I did not double check the ReadOnlySessionStore interface before, and
just assumed it would take a timestamp, too. My bad.
Please update the KIP for ReadOnlyWindowStore and WindowStore.
I like the KIP as-is. Feel free to start a VOTE thread. Even if there
might be minor follow up comments
Hi Nikolay,
Yes, the changes we discussed for ReadOnlyXxxStore and XxxStore should be
in this KIP.
And you're right, it seems like ReadOnlySessionStore is not necessary to
touch, since it doesn't reference any `long` timestamps.
Thanks,
-John
On Wed, Sep 12, 2018 at 4:36 AM Nikolay Izhikov wro
Hello, Matthias.
> His proposal is, to deprecate existing methods on `ReadOnlyWindowStore`> and
> `ReadOnlySessionStore` and add them to `WindowStore` and> `SessionStore`
> Does this make sense?
You both are experienced Kafka developers, so yes, it does make a sense to me
:).
Do we want to make
I talked to John offline about his last suggestions, that I originally
did not fully understand.
His proposal is, to deprecate existing methods on `ReadOnlyWindowStore`
and `ReadOnlySessionStore` and add them to `WindowStore` and
`SessionStore` (note, all singular -- not to be confused with classe
Hello, Guozhang, Bill.
> 1) I'd suggest keeping `Punctuator#punctuate(long timestamp)` as is
I am agree with you.
Currently, `Punctuator` edits are not included in KIP.
> 2) I'm fine with keeping KeyValueStore extending ReadOnlyKeyValueStore
Great, currently, there is no suggested API change in
Hi Nikolay,
I'm a +1 to points 1 and 2 above from Guozhang.
Thanks,
Bill
On Mon, Sep 10, 2018 at 6:58 PM Guozhang Wang wrote:
> Hello Nikolay,
>
> Thanks for picking this up! Just sharing my two cents:
>
> 1) I'd suggest keeping `Punctuator#punctuate(long timestamp)` as is since
> comparing wi
Hello Nikolay,
Thanks for picking this up! Just sharing my two cents:
1) I'd suggest keeping `Punctuator#punctuate(long timestamp)` as is since
comparing with other places where we are replacing with Duration and
Instant, this is not a user specified value as part of the DSL but rather a
passed-i
Hello, Matthias.
> (4) While I agree that we might want to deprecate it, I am not sure if this
> should be part of this KIP?
> Seems to be unrelated?
> Should this have been part of KIP-319?
> If yes, we might still want to updated this other KIP? WDYT?
OK, I removed this deprecation from thi
(1) Sounds good to me, to just use IllegalArgumentException for both --
and thanks for pointing out that Duration can be negative and we need to
check for this. For the KIP, it would be nice to add to all methods than
(even if we don't do it in the code but only document in JavaDocs).
(2) I would
Hey all,
(1): Duration can be negative, just like long. We need to enforce any
bounds that we currently enforce. We don't need the `throws` declaration
for runtime exceptions, but the potential IllegalArgumentException should
be documented in the javadoc for these methods. I still feel that surfac
Hello, Matthias.
Thanks, for feedback.
> (1) Some methods declare `throws IllegalArgumentException`, others> don't.
`duration.toMillis()` can throw ArithmeticException.
It can happen if overflow occurs during conversion.
Please, see source of jdk method Duration#toMillis.
Task author suggest to
Thanks for updating the KIP!
Couple of minor follow ups:
(1) Some methods declare `throws IllegalArgumentException`, others
don't. It's runtime exception and thus it's not required to declare it
-- it just looks inconsistent in the KIP and maybe it's inconsistent in
the code, too. I am not sure i
Hello, Guys.
I've started a VOTE [1], but seems commiters have no chance to look at KIP for
now.
Can you tell me, is it OK?
Should I wait for feedback? For how long?
Or something in KIP should be improved before voting?
[1]
https://lists.apache.org/thread.html/e976352e7e42d459091ee66ac790b6a0
Hello,
I want to start VOTE for this KIP today.
Any objections?
В Пн, 27/08/2018 в 10:20 +0300, Nikolay Izhikov пишет:
> Hello, Matthias, John.
>
> Thanks in advance.
>
> > I wanted to let you know that we have dropped the `grace(long)` method from
> > the Windows interface
>
> `grace(long)`
Hello, Matthias, John.
Thanks in advance.
> I wanted to let you know that we have dropped the `grace(long)` method from
> the Windows interface
`grace(long)` removed from the KIP.
> It seems like, if we want to use long millis internally, then we just need to
> leave Windows alone.
`Windows`
It's tricky... :)
Some APIs have "dual use" as I mentioned in my first reply. I agree that
it would be good to avoid abstract class and use interfaces if possible.
As long as the change is source code compatible, it should be fine IMHO
-- we need to document binary incompatibility of course.
I th
Quick afterthought: I guess that `Window` is exposed to the API via
`Windowed` keys. I think it would be fine to not deprecate the `long` start
and end, but add `Instant` variants for people preferring that interface.
On Fri, Aug 24, 2018 at 11:10 AM John Roesler wrote:
> Hey Matthias,
>
> Thank
Hey Matthias,
Thanks for pointing that out. I agree that we only really need to change
methods that are API-facing, and we probably want to avoid using
Duration/Instant for Streams-facing members.
Like I said in my last email, I think the whole Windows interface is
Streams-facing, and the builder
Hi Nikolay,
First: I wanted to let you know that we have dropped the `grace(long)`
method from the Windows interface, but we do still need to transition the
same method on TimeWindows and JoinWindows (
https://github.com/apache/kafka/pull/5536)
I have also been thinking it would be nice to replac
Hello, Mathias.
Thanks for your feedback.
> Thus, it might make sense to keep old and just add new ones?
As far as I understand, we will keep old methods anyway to prevent public API
backward compatibility.
I agree with you, methods that used internally shouldn't be deprecated.
> End users ca
Thanks a lot for the KIP.
From my understanding, the idea of the KIP is to improve the public API
at DSL level. However, not all public methods listed are part of DSL
level API, but part of runtime API. Those methods are called during
processing and are on the hot code path. I am not sure, if we w
Dear, commiters.
Please, pay attention to this KIP and share your opinion.
В Вт, 21/08/2018 в 11:14 -0500, John Roesler пишет:
> I'll solicit more reviews. Let's get at least one committer to chime in
> before we start a vote (since we need their approval anyway).
> -John
>
> On Mon, Aug 20, 201
I'll solicit more reviews. Let's get at least one committer to chime in
before we start a vote (since we need their approval anyway).
-John
On Mon, Aug 20, 2018 at 12:39 PM Nikolay Izhikov
wrote:
> Hello, Ted.
>
> Thanks for the comment.
>
> I've edit KIP and change proposal to `windowSize`.
>
>
Hello, Ted.
Thanks for the comment.
I've edit KIP and change proposal to `windowSize`.
Guys, any other comments?
В Вс, 19/08/2018 в 14:57 -0700, Ted Yu пишет:
> bq. // or just Duration windowSize();
>
> +1 to the above choice.
> The duration is obvious from the return type. For getter methods
bq. // or just Duration windowSize();
+1 to the above choice.
The duration is obvious from the return type. For getter methods, we don't
use get as prefix (as least for new code).
Cheers
On Sun, Aug 19, 2018 at 8:03 AM Nikolay Izhikov wrote:
> Hello, John.
>
> Thank you very much for your feed
Hello, John.
Thank you very much for your feedback!
I've addressed all your comments.
Please, see my answers and let my know is anything in KIP [1] needs to be
improved.
> The correct choice is actually "Instant", not> "LocalDateTime"
I've changed the methods proposed in KIP [1] to use Instant.
Hi Nikolay,
Thanks for this very nice KIP!
To answer your questions:
1. Correct, we should not delete existing methods that have been released,
but ...
2. Yes, we should deprecate the 'long' variants so that we can drop them
later on. Personally, I like to mention which version deprecated the me
Hello, Kafka developers.
I would like to start a discussion of KIP-358 [1].
It based on a ticket KAFKA-7277 [2].
I crawled through Stream API and made my suggestions for API changes.
I have several questions about changes.
Please, share your comments:
1. I propose do not remove existing API met
33 matches
Mail list logo