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 if it is possible to provide a negative
Duration? If not, we would not need to check the provided value and can
remove the declaration.

(2) About punctuations: I still think, it would be ok to change the
callback from `long` to `Instance` -- even if it is possible to register
a punctuation on a ms-basis, in practice many people used schedules in
the range of seconds or larger. Thus, I don't think there will be a
performance penalty. Of course, we can still revisit this later, too.
John and Bill, you did not comment on this. Would also be good to get
feedback from Guozhang about this.

(3) ReadOnlyWindowStore: All three methods are dual use and I think we
should not deprecate them. However, we can add the new proposed methods
in parallel -- the names can be the same without conflict as the
parameter lists are different. (Or did you just forget to remove the
comment line?)

(4) Stores: 3 methods are listed as deprecated but only 2 new methods
are added. Maybe this was discussed already, but I can't recall why? Can
you elaborate? Or should this deprecation be actually be part of KIP-328
(\cc John)?


Thanks,

-Matthias



ps: there are many KIPs in-flight in parallel, and it takes some time to
get around. Please be patient :)




On 9/5/18 12:25 AM, Nikolay Izhikov wrote:
> 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/e976352e7e42d459091ee66ac790b6a0de7064eac0c57760d50c983b@%3Cdev.kafka.apache.org%3E
> 
> В Пт, 24/08/2018 в 10:36 -0700, Matthias J. Sax пишет:
>> 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 think it's best, if the KIPs gets update with a proposal on how to
>> handle "dual use" parts. It's easier to discuss if it's written down IMHO.
>>
>> For `ProcessorContext#schedule()`, you are right John: it's seems fine
>> to use `Duration`, as it won't be called often (usually only within
>> `Processor#init()`) -- I mixed it up with `Punctuator#punctuate(long)`.
>> However, thinking about this twice, we might even want to update both
>> methods. Punctuation callbacks don't happen every millisecond and thus
>> the overhead to use `Instance` should not be a problem.
>>
>> @Nikolay: it seems the KIP does not mention `Punctuator#punctuate(long)`
>> -- should we add it?
>>
>>
>> -Matthias
>>
>>
>> On 8/24/18 10:11 AM, John Roesler wrote:
>>> 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 <j...@confluent.io> wrote:
>>>
>>>> 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 builders we provide are otherwise API-facing.
>>>> Likewise, `Window` is Streams-facing, so start and end should not use
>>>> Duration. In SessionWindows, inactivityGap is Streams-facing.
>>>>
>>>> I actually think that ProcessorContext#schedule() is API-facing, so it
>>>> should use Duration. The rationale is that streams processing doesn't call
>>>> this method, only implementer of Processor do. Does that seem right?
>>>>
>>>> Also, it seems like  ReadOnlyWindowStore#fetch() (2x) and #fetchAll() are
>>>> API-facing (for IQ). When we call fetch() during processing, it's actually
>>>> `WindowStore#fetch()`. Maybe we should move "WindowStoreIterator<V> fetch(K
>>>> key, long timeFrom, long timeTo)" to the WindowStore interface and make
>>>> all the ReadOnlyWindowStore methods take Durations. And likewise with the
>>>> SessionStore interfaces.
>>>>
>>>> What do you think?
>>>>
>>>> Thanks,
>>>> -John
>>>>
>>>>
>>>>
>>>>
>>>> On Fri, Aug 24, 2018 at 10:51 AM John Roesler <j...@confluent.io> wrote:
>>>>
>>>>> 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 replace `Windows` with an
>>>>> interface, but for different reasons. I think we can even do it without
>>>>> breaking source compatibility (but it would break binary compatibility):
>>>>> create a new interface `WindowSpec`, deprecate `Windows` and make it
>>>>> implement `WindowSpec`, add a new method:
>>>>> `KGroupedStream#windowedBy(WindowSpec)`, and deprecate the old one.
>>>>>
>>>>> However, I don't think this would solve your problem, since the Windows
>>>>> interface has two audiences: the DSL user and the implementer who wishes 
>>>>> to
>>>>> provide a new kind of windowing. I think we want to provide Duration to 
>>>>> the
>>>>> former, and long or Duration is fine for the latter. However, both of 
>>>>> these
>>>>> audiences are "external", so having an "internal" interface won't fit the
>>>>> bill.
>>>>>
>>>>> I think my last PR #5536 actually helps the situation quite a bit. Let's
>>>>> forget about the deprecated members. Now, all the public members of 
>>>>> Windows
>>>>> are abstract methods, so Windows is effectively an interface now. Here's
>>>>> how it looks:
>>>>>
>>>>> public abstract class Windows<W extends Window> {
>>>>> public abstract Map<Long, W> windowsFor(final long timestamp);
>>>>> public abstract long size();
>>>>> public abstract long gracePeriodMs();
>>>>> }
>>>>>
>>>>> Notice that there is no part of this involved with the DSL. When you're
>>>>> writing a topology, you don't call any of these methods. It's strictly an
>>>>> interface that tells a Windows implementation what Streams expects from 
>>>>> it.
>>>>> A very simple implementation could have no builder methods at all and just
>>>>> return fixed answers to these method calls (this is basically what
>>>>> UnlimitedWindows does). It seems like, if we want to use long millis
>>>>> internally, then we just need to leave Windows alone.
>>>>>
>>>>> What we do want to change is the builder methods in TimeWindows,
>>>>> JoinWindows, and UnlimitedWindows. For example, `TimeWindows#of(long)`
>>>>> would become `TimeWindows#of(Duration)`, etc. These are the DSL methods.
>>>>>
>>>>> Does that make sense?
>>>>> -John
>>>>>
>>>>>
>>>>>
>>>>> On Thu, Aug 23, 2018 at 8:59 AM Nikolay Izhikov <nizhi...@apache.org>
>>>>> wrote:
>>>>>
>>>>>> 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 can use the "nicer" new ones, while we can still use the
>>>>>>
>>>>>> existing ones internally?
>>>>>>> Not sure if it would be possible to keep the old ones without exposing
>>>>>>
>>>>>> them as public API?
>>>>>>
>>>>>> I think, when we decide to remove methods with `long` from public API,
>>>>>> we can do the following:
>>>>>>
>>>>>> 1. Create an interface like `WindowsInternal`.
>>>>>> 2. Change Windows to an interface.
>>>>>> 3. Create package-private implementation `WindowsImpl`.
>>>>>>
>>>>>> ```
>>>>>>         package org.apache.kafka.streams.kstream.internals;
>>>>>>         public interface WindowsInternal {
>>>>>>                 public long start();
>>>>>>                 public long end();
>>>>>>                 //etc...
>>>>>>         }
>>>>>>
>>>>>>         package org.apache.kafka.streams.kstream;
>>>>>>         public interface Windows<W extends Window> {
>>>>>>                 public Instant start();
>>>>>>                 public Instant end();
>>>>>>                 //...
>>>>>>         }
>>>>>>
>>>>>>         class WindowsImpl<W extends Window> implements Windows<W>,
>>>>>> WindowsInternal {
>>>>>>
>>>>>>         }
>>>>>> ```
>>>>>>
>>>>>> So, in public API we will expose only `Windows` interface and internally
>>>>>> we can use `WindowsInternal`
>>>>>> But, of course, this will be huge changes in public API.
>>>>>>
>>>>>>> Let me know what you think about this.
>>>>>>
>>>>>> I think in this KIP we shouldn't deprecate methods, that are used
>>>>>> internally.
>>>>>> I changed it, now my proposal is just add new methods.
>>>>>>
>>>>>> Please, let me know if anything more need to be done.
>>>>>>
>>>>>> В Ср, 22/08/2018 в 17:29 -0700, Matthias J. Sax пишет:
>>>>>>> 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 want to
>>>>>>> update those methods. We should carefully think about this, and
>>>>>>
>>>>>> consider
>>>>>>> to keep Long/long type to keep runtime overhead small. Note, that the
>>>>>>> methods I mention are not required to specify a program using the DSL
>>>>>>> and thus is questionable if the DSL API would be improved if we change
>>>>>>> the methods.
>>>>>>>
>>>>>>> It's unfortunate, that some part of the public API stretch the DSL
>>>>>>> builder part as well as the runtime part...
>>>>>>>
>>>>>>> This affects the following methods (please double check if I missed
>>>>>>
>>>>>> any):
>>>>>>>
>>>>>>>  - Windows#windowsFor()
>>>>>>>  - Window#start()
>>>>>>>  - Window#end()
>>>>>>>  - JoinWindows#windowFor()
>>>>>>>  - SessionWindows#inactivitiyGap()
>>>>>>>  - TimeWindows#windowFor()
>>>>>>>  - UnlimitedWindows#windowFor()
>>>>>>>  - ProcessorContext#schedule()
>>>>>>>  - ReadOnlyWindowStore#fetch() (2x) and #fetchAll()
>>>>>>>  - SessionStore#findSessions() (2x)
>>>>>>>
>>>>>>> maybe
>>>>>>>  - TimeWindowedDeserializer#getWindowSize() (it's unused atm, but I
>>>>>>> could imagine that it might be use on the hot code path in the furture)
>>>>>>>
>>>>>>> So methods have "dual" use and might be called externally and
>>>>>>
>>>>>> internally:
>>>>>>>
>>>>>>>  - Window#start()
>>>>>>>  - Window#end()
>>>>>>>  - ReadOnlyWindowStore#fetch() (2x) and #fetchAll()
>>>>>>>  - SessionStore#findSessions() (2x)
>>>>>>>
>>>>>>> Thus, it might make sense to keep old and just add new ones? End users
>>>>>>> can use the "nicer" new ones, while we can still use the existing ones
>>>>>>> internally? Not sure if it would be possible to keep the old ones
>>>>>>> without exposing them as public API?
>>>>>>>
>>>>>>> Let me know what you think about this.
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 8/21/18 11:41 PM, Nikolay Izhikov wrote:
>>>>>>>> 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, 2018 at 12:39 PM Nikolay Izhikov <
>>>>>>
>>>>>> nizhi...@apache.org>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> 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, we
>>>>>>>>>>
>>>>>>>>>> don't
>>>>>>>>>>> use get as prefix (as least for new code).
>>>>>>>>>>>
>>>>>>>>>>> Cheers
>>>>>>>>>>>
>>>>>>>>>>> On Sun, Aug 19, 2018 at 8:03 AM Nikolay Izhikov <
>>>>>>
>>>>>> nizhi...@apache.org>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> 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.
>>>>>>>>>>>>
>>>>>>>>>>>>> I noticed some recent APIs are> missing (see KIP-328)
>>>>>>>>>>>>> those APIs were just added and have never been released...
>>>>>>
>>>>>> you can
>>>>>>>>>>
>>>>>>>>>> just
>>>>>>>>>>>>
>>>>>>>>>>>> replace them.
>>>>>>>>>>>>
>>>>>>>>>>>> I've added new methods to KIP [1].
>>>>>>>>>>>> Not released methods marked for remove.
>>>>>>>>>>>>
>>>>>>>>>>>>> any existing method that's already deprecated, don't bother
>>>>>>>>>>>>
>>>>>>>>>>>> transitioning it to Duration.
>>>>>>>>>>>>
>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>
>>>>>>>>>>>>> IllegalArgumentException... we should plan to mention this
>>>>>>
>>>>>> in the
>>>>>>>>>>>>
>>>>>>>>>>>> javadoc for those methods.
>>>>>>>>>>>>
>>>>>>>>>>>> Got it.
>>>>>>>>>>>>
>>>>>>>>>>>>> In Stores, windowSize and segmentInterval should also be
>>>>>>
>>>>>> durations.
>>>>>>>>>>>>
>>>>>>>>>>>> Fixed.
>>>>>>>>>>>>
>>>>>>>>>>>>> StreamsMetrics, recordLatency ... this one is better left
>>>>>>
>>>>>> alone.
>>>>>>>>>>>>
>>>>>>>>>>>> OK. I removed this method from KIP [1].
>>>>>>>>>>>>
>>>>>>>>>>>> Two more questions question about implementation:
>>>>>>>>>>>>
>>>>>>>>>>>> 1. We have serveral methods without parameters.
>>>>>>>>>>>> In java we can't have two methods with parameters with the
>>>>>>
>>>>>> same name.
>>>>>>>>>>>> It wouldn't compile.
>>>>>>>>>>>> So we have to rename new methods. Please, see suggested
>>>>>>
>>>>>> names and share
>>>>>>>>>>>> your thoughts:
>>>>>>>>>>>>
>>>>>>>>>>>> Windows {
>>>>>>>>>>>>     long size() -> Duration windowSize();
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> Window {
>>>>>>>>>>>>     long start() -> Instant startTime();
>>>>>>>>>>>>     long end() -> Instant endTime();
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> SessionWindows {
>>>>>>>>>>>>     long inactivityGap() -> Duration inactivityGapDuration();
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> TimeWindowedDeserializer {
>>>>>>>>>>>>     Long getWindowSize() -> Duration
>>>>>>
>>>>>> getWindowSizeDuration(); // or
>>>>>>>>>>
>>>>>>>>>> just
>>>>>>>>>>>> Duration windowSize();
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> SessionBytesStoreSupplier {
>>>>>>>>>>>>     long retentionPeriod() -> Duration
>>>>>>
>>>>>> retentionPeriodDuration();
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> WindowBytesStoreSupplier {
>>>>>>>>>>>>     long windowSize() -> Duration windowSizeDuration();
>>>>>>>>>>>>     long retentionPeriod() -> Duration
>>>>>>
>>>>>> retentionPeriodDuration();
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> 2. Do we want to use Duration and Instant inside API
>>>>>>
>>>>>> implementations?
>>>>>>>>>>>>
>>>>>>>>>>>> IGNITE-7277: "Durations potentially worsen memory pressure
>>>>>>
>>>>>> and gc
>>>>>>>>>>>> performance, so internally, we will still use longMs as the
>>>>>>>>>>
>>>>>>>>>> representation."
>>>>>>>>>>>> IGNITE-7222: Duration used to store retention.
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-358%3A+Migrate+Streams+API+to+Duration+instead+of+long+ms+times
>>>>>>>>>>>> [2]
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>
>>>>>> https://github.com/apache/kafka/commit/b3771ba22acad7870e38ff7f58820c5b50946787#diff-47289575d3e3e2449f27b3a7b6788e1aR64
>>>>>>>>>>>>
>>>>>>>>>>>> В Пт, 17/08/2018 в 14:46 -0500, John Roesler пишет:
>>>>>>>>>>>>> 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
>>>>>>>>>>>>
>>>>>>>>>>>> method
>>>>>>>>>>>>> so everyone can see later on how long it's been
>>>>>>
>>>>>> deprecated, but this
>>>>>>>>>>
>>>>>>>>>> may
>>>>>>>>>>>>
>>>>>>>>>>>> be
>>>>>>>>>>>>> controversial, so let's let other weigh in.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3. I think you're asking whether it's appropriate to drop
>>>>>>
>>>>>> the "Ms"
>>>>>>>>>>>>
>>>>>>>>>>>> suffix,
>>>>>>>>>>>>> and I think yes. So "long inactivityGapMs" would become
>>>>>>
>>>>>> "Duration
>>>>>>>>>>>>> inactivityGap".
>>>>>>>>>>>>> In the places where the parameter's name is just
>>>>>>
>>>>>> "duration", I think
>>>>>>>>>>
>>>>>>>>>> we
>>>>>>>>>>>>
>>>>>>>>>>>> can
>>>>>>>>>>>>> pick something more descriptive (I realize it was already
>>>>>>>>>>
>>>>>>>>>> "durationMs";
>>>>>>>>>>>>> this is just a good time to improve it).
>>>>>>>>>>>>> Also, you're correct that we shouldn't use a Duration to
>>>>>>
>>>>>> represent a
>>>>>>>>>>>>
>>>>>>>>>>>> moment
>>>>>>>>>>>>> in time, like "startTime". The correct choice is actually
>>>>>>
>>>>>> "Instant",
>>>>>>>>>>
>>>>>>>>>> not
>>>>>>>>>>>>> "LocalDateTime", though.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>
>>>>>> https://stackoverflow.com/questions/32437550/whats-the-difference-between-instant-and-localdatetime
>>>>>>>>>>>>> explains why.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I also had a few notes on the KIP itself:
>>>>>>>>>>>>> 4. You might want to pull trunk again. I noticed some
>>>>>>
>>>>>> recent APIs are
>>>>>>>>>>>>> missing (see KIP-328).
>>>>>>>>>>>>>
>>>>>>>>>>>>> 5. Speaking of KIP-328: those APIs were just added and
>>>>>>
>>>>>> have never
>>>>>>>>>>
>>>>>>>>>> been
>>>>>>>>>>>>> released, so there's no need to deprecate the methods, you
>>>>>>
>>>>>> can just
>>>>>>>>>>>>
>>>>>>>>>>>> replace
>>>>>>>>>>>>> them.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 6. For any existing method that's already deprecated,
>>>>>>
>>>>>> don't bother
>>>>>>>>>>>>> transitioning it to Duration. I think the examples I
>>>>>>
>>>>>> noticed were
>>>>>>>>>>>>> deprecated in KIP-328, so you'll see what I'm talking
>>>>>>
>>>>>> about when you
>>>>>>>>>>
>>>>>>>>>> pull
>>>>>>>>>>>>> trunk again.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 7. Any method taking a Duration argument may throw an
>>>>>>>>>>>>> IllegalArgumentException (we choose to convert
>>>>>>
>>>>>> ArithmeticException to
>>>>>>>>>>>>> IllegalArgumentException, as I mentioned in the Jira
>>>>>>
>>>>>> ticket). We
>>>>>>>>>>
>>>>>>>>>> don't
>>>>>>>>>>>>
>>>>>>>>>>>> need
>>>>>>>>>>>>> a "throws" declaration, but we should plan to mention this
>>>>>>
>>>>>> in the
>>>>>>>>>>
>>>>>>>>>> javadoc
>>>>>>>>>>>>> for those methods.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 8. In Stores, windowSize and segmentInterval should also be
>>>>>>>>>>
>>>>>>>>>> durations.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 9. In StreamsMetrics, recordLatency could be just a
>>>>>>
>>>>>> Duration, but I
>>>>>>>>>>>>> actually think this one is better left alone. IMO, it's
>>>>>>
>>>>>> more effort
>>>>>>>>>>
>>>>>>>>>> for
>>>>>>>>>>>>> little gain to require users to construct a Duration
>>>>>>
>>>>>> before they
>>>>>>>>>>
>>>>>>>>>> call the
>>>>>>>>>>>>> method, since they vary likely call
>>>>>>
>>>>>> System.currentTimeNanos before
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>>>> after the code in question.
>>>>>>>>>>>>>
>>>>>>>>>>>>> These are quite a few notes, but they're all minor.
>>>>>>
>>>>>> Overall the KIP
>>>>>>>>>>
>>>>>>>>>> looks
>>>>>>>>>>>>> really good to me. Thanks for picking this up!
>>>>>>>>>>>>> -John
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 16, 2018 at 9:55 AM Nikolay Izhikov <
>>>>>>
>>>>>> nizhi...@apache.org
>>>>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> 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 methods with
>>>>>>
>>>>>> long ms
>>>>>>>>>>>>
>>>>>>>>>>>> parameters.
>>>>>>>>>>>>>> Is it correct?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2. Should we mark existing methods as deprecated?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 3. Suggested changes in ticket description are `long
>>>>>>
>>>>>> durationMs` to
>>>>>>>>>>>>>> `Duration duration` and similar.
>>>>>>>>>>>>>> I suggest to change 'long startTimeMs` to `LocalDateTime
>>>>>>
>>>>>> startTime`
>>>>>>>>>>>>
>>>>>>>>>>>> also.
>>>>>>>>>>>>>> Should we do it?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Please, note, it very first KIP for me, so tell me if I
>>>>>>
>>>>>> miss
>>>>>>>>>>
>>>>>>>>>> something
>>>>>>>>>>>>>> obvious for experienced Kafka developers.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-358%3A+Migrate+Streams+API+to+Duration+instead+of+long+ms+times
>>>>>>>>>>>>>> [2] https://issues.apache.org/jira/browse/KAFKA-7277
>>>>>>>
>>>>>>>
>>>>>
>>>>>
>>

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to