You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Michal Borowiecki <mi...@openbet.com> on 2017/04/03 09:34:07 UTC

[DISCUSS] KIP-138: Change punctuate semantics

Hi all,

I have created a draft for KIP-138: Change punctuate semantics
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%3A+Change+punctuate+semantics>
.

Appreciating there can be different views on system-time vs event-time
semantics for punctuation depending on use-case and the importance of
backwards compatibility of any such change, I've left it quite open and
hope to fill in more info as the discussion progresses.

Thanks,
Michal

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Damian Guy <da...@gmail.com>.
Thanks Michael  - LGTM

On Fri, 5 May 2017 at 12:04 Michal Borowiecki <mi...@openbet.com>
wrote:

> I shall move all alternatives other than the main proposal into the
> Rejected Alternatives section and if I hear any objections, I'll move those
> back up and we'll discuss further.
>
> Done.
>
>
> Still looking forward to any comments, especially about the recently
> proposed ability to cancel punctuation schedules. I think it goes well in
> the spirit of making complex things possible (such as the hybrid semantics).
>
>
> In the absence of further comments I shall call for a vote in the next few
> days.
>
>
> Thanks,
>
> Michał
>
> On 04/05/17 09:41, Michal Borowiecki wrote:
>
> Further in this direction I've updated the main proposal to incorporate
> the Cancellable return type for ProcessorContext.schedule and the guidance
> on how to implement "hybrid" punctuation with the proposed 2
> PunctuationTypes.
>
> I look forward to more comments whether the Cancallable return type is an
> agreeable solution and it's precise definition.
>
> I shall move all alternatives other than the main proposal into the
> Rejected Alternatives section and if I hear any objections, I'll move those
> back up and we'll discuss further.
>
>
> Looking forward to all comments and suggestions.
>
>
> Thanks,
>
> Michal
>
> On 01/05/17 18:23, Michal Borowiecki wrote:
>
> Hi all,
>
> As promised, here is my take at how one could implement the previously
> discussed hybrid semantics using the 2 PunctuationType callbacks (one for
> STREAM_TIME and one for SYSTEM_TIME).
>
> However, there's a twist.
>
> Since currently calling context.schedule() adds a new PunctuationSchedule
> and does not overwrite the previous one, a slight change would be required:
>
> a) either that PuncuationSchedules are cancellable
>
> b) or that calling schedule() overwrites(cancels) the previous one with
> the given PunctuationType (but that's not how it works currently)
>
>
> Below is an example assuming approach a) is implemented by having schedule
> return Cancellable instead of void.
> ProcessorContext context;
> long streamTimeInterval = ...;
> long systemTimeUpperBound = ...; //e.g. systemTimeUpperBound =
> streamTimeInterval + some tolerance
> Cancellable streamTimeSchedule;
> Cancellable systemTimeSchedule;
> long lastStreamTimePunctation = -1;
>
> public void init(ProcessorContext context){
>     this.context = context;
>     streamTimeSchedule = context.schedule(PunctuationType.STREAM_TIME,
> streamTimeInterval,   this::streamTimePunctuate);
>     systemTimeSchedule = context.schedule(PunctuationType.SYSTEM_TIME,
> systemTimeUpperBound, this::systemTimePunctuate);
> }
>
> public void streamTimePunctuate(long streamTime){
>     periodicBusiness(streamTime);
>
>     systemTimeSchedule.cancel();
>     systemTimeSchedule = context.schedule(PunctuationType.SYSTEM_TIME,
> systemTimeUpperBound, this::systemTimePunctuate);
> }
>
> public void systemTimePunctuate(long systemTime){
>     periodicBusiness(context.timestamp());
>
>     streamTimeSchedule.cancel();
>     streamTimeSchedule = context.schedule(PunctuationType.STREAM_TIME,
> streamTimeInterval, this::streamTimePunctuate);
> }
>
> public void periodicBusiness(long streamTime){
>     // guard against streamTime == -1, easy enough.
>     // if you need system time instead, just use
> System.currentTimeMillis()
>
>     // do something businessy here
> }
>
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
> I shall move all alternatives other than the main proposal into the 
> Rejected Alternatives section and if I hear any objections, I'll move 
> those back up and we'll discuss further.
Done.


Still looking forward to any comments, especially about the recently 
proposed ability to cancel punctuation schedules. I think it goes well 
in the spirit of making complex things possible (such as the hybrid 
semantics).


In the absence of further comments I shall call for a vote in the next 
few days.


Thanks,

Michał


On 04/05/17 09:41, Michal Borowiecki wrote:
>
> Further in this direction I've updated the main proposal to 
> incorporate the Cancellable return type for ProcessorContext.schedule 
> and the guidance on how to implement "hybrid" punctuation with the 
> proposed 2 PunctuationTypes.
>
> I look forward to more comments whether the Cancallable return type is 
> an agreeable solution and it's precise definition.
>
> I shall move all alternatives other than the main proposal into the 
> Rejected Alternatives section and if I hear any objections, I'll move 
> those back up and we'll discuss further.
>
>
> Looking forward to all comments and suggestions.
>
>
> Thanks,
>
> Michal
>
>
> On 01/05/17 18:23, Michal Borowiecki wrote:
>>
>> Hi all,
>>
>> As promised, here is my take at how one could implement the 
>> previously discussed hybrid semantics using the 2 PunctuationType 
>> callbacks (one for STREAM_TIME and one for SYSTEM_TIME).
>>
>> However, there's a twist.
>>
>> Since currently calling context.schedule() adds a new 
>> PunctuationSchedule and does not overwrite the previous one, a slight 
>> change would be required:
>>
>> a) either that PuncuationSchedules are cancellable
>>
>> b) or that calling schedule() ||overwrites(cancels) the previous one 
>> with the given |PunctuationType |(but that's not how it works currently)
>>
>>
>> Below is an example assuming approach a) is implemented by having 
>> schedule return Cancellable instead of void.
>>
>> |ProcessorContext context;|
>> |long||streamTimeInterval = ...;|
>> |long||systemTimeUpperBound = ...;||//e.g. systemTimeUpperBound = 
>> streamTimeInterval + some tolerance|
>> |Cancellable streamTimeSchedule;|
>> |Cancellable systemTimeSchedule;|
>> |long||lastStreamTimePunctation = -||1||;|
>> ||
>> |public||void||init(ProcessorContext context){|
>> |||this||.context = context;|
>> |||streamTimeSchedule = context.schedule(PunctuationType.STREAM_TIME, 
>> streamTimeInterval, ||this||::streamTimePunctuate);|
>> |||systemTimeSchedule = context.schedule(PunctuationType.SYSTEM_TIME, 
>> systemTimeUpperBound,||this||::systemTimePunctuate); |
>> |}|
>> ||
>> |public||void||streamTimePunctuate(||long||streamTime){|
>> |||periodicBusiness(streamTime);|
>> |||systemTimeSchedule.cancel();|
>> |||systemTimeSchedule = context.schedule(PunctuationType.SYSTEM_TIME, 
>> systemTimeUpperBound,||this||::systemTimePunctuate);|
>> |}|
>> ||
>> |public||void||systemTimePunctuate(||long||systemTime){|
>> |||periodicBusiness(context.timestamp());|
>> |||streamTimeSchedule.cancel();|
>> |||streamTimeSchedule = context.schedule(PunctuationType.STREAM_TIME, 
>> streamTimeInterval,||this||::streamTimePunctuate);|
>> |}|
>> ||
>> |public||void||periodicBusiness(||long||streamTime){|
>> |||// guard against streamTime == -1, easy enough.|
>> |||// if you need system time instead, just use 
>> System.currentTimeMillis()|
>> ||
>> |||// do something businessy here|
>> |}|
>>
>> Where Cancellable is either an interface containing just a single 
>> void cancel() method or also boolean isCancelled() likehere 
>> <http://doc.akka.io/japi/akka/2.5.0/akka/actor/Cancellable.html>.
>>
>>
>> Please let your opinions known whether we should proceed in this 
>> direction or leave "hybrid" considerations out of scope.
>>
>> Looking forward to hearing your thoughts.
>>
>> Thanks,
>> Michal
>>
>> On 30/04/17 20:07, Michal Borowiecki wrote:
>>>
>>> Hi Matthias,
>>>
>>> I'd like to start moving the discarded ideas into Rejected 
>>> Alternatives section. Before I do, I want to tidy them up, ensure 
>>> they've each been given proper treatment.
>>>
>>> To that end let me go back to one of your earlier comments about the 
>>> original suggestion (A) to put that to bed.
>>>
>>>
>>> On 04/04/17 06:44, Matthias J. Sax wrote:
>>>> (A) You argue, that users can still "punctuate" on event-time via
>>>> process(), but I am not sure if this is possible. Note, that users only
>>>> get record timestamps via context.timestamp(). Thus, users would need to
>>>> track the time progress per partition (based on the partitions they
>>>> obverse via context.partition(). (This alone puts a huge burden on the
>>>> user by itself.) However, users are not notified at startup what
>>>> partitions are assigned, and user are not notified when partitions get
>>>> revoked. Because this information is not available, it's not possible to
>>>> "manually advance" stream-time, and thus event-time punctuation within
>>>> process() seems not to be possible -- or do you see a way to get it
>>>> done? And even if, it might still be too clumsy to use.
>>> I might have missed something but I'm guessing your worry about 
>>> users having to track time progress /per partition/ comes from the 
>>> what the stream-time does currently.
>>> But I'm not sure that those semantics of stream-time are ideal for 
>>> users of punctuate.
>>> That is, if stream-time punctuate didn't exist and users had to use 
>>> process(), would they actually want to use the current semantics of 
>>> stream time?
>>>
>>> As a reminder stream time, in all its glory, is (not exactly 
>>> actually, but when trying to be absolutely precise here I spotted 
>>> KAFKA-5144 <https://issues.apache.org/jira/browse/KAFKA-5144> so I 
>>> think this approximation suffices to illustrate the point):
>>>
>>> a minimum across all input partitions of (
>>>    if(msgs never received by partition) -1;
>>>    else {
>>>       a non-descending-minimum of ( the per-batch minimum msg 
>>> timestamp)
>>>    }
>>> )
>>>
>>> Would that really be clear enough to the users of punctuate? Do they 
>>> care for such a convoluted notion of time? I see how this can be 
>>> useful for StreamTask to pick the next partition to take a record 
>>> from but for punctuate?
>>> If users had to implement punctuation with process(), is that what 
>>> they would have chosen as their notion of time?
>>> I'd argue not.
>>>
>>> None of the processors implementing the rich windowing/join 
>>> operations in the DSL use punctuate.
>>> Let's take the KStreamKStreamJoinProcessor as an example, in it's 
>>> process() method it simply uses context().timestamp(), which, since 
>>> it's called from process, returns simply, per javadoc:
>>> If it is triggered while processing a record streamed from the 
>>> source processor, timestamp is defined as the timestamp of the 
>>> current input record;
>>> So they don't use that convoluted formula for stream-time. Instead, 
>>> they only care about the timestamp of the current record. I think 
>>> that having users track just that wouldn't be that much of a burden. 
>>> I don't think they need to care about which partitions got assigned 
>>> or not. And StreamTask would still be picking records first from the 
>>> partition having the lowest timestamp to try to "synchronize" the 
>>> streams as it does now.
>>>
>>> What users would have to do in their Processor implementations is 
>>> somewhere along the lines of:
>>>
>>> long lastPunctuationTime = 0;
>>> long interval = <some-number>; //millis
>>>
>>> @Override
>>> public void process(K key, V value){
>>>     while (ctx.timestamp() >= lastPunctuationTime + interval){
>>>         punctuate(ctx.timestamp());
>>>         lastPunctuationTime += interval;// I'm not sure of the merit 
>>> of this vs lastPunctuationTime = ctx.timestamp(); but that's what 
>>> PunctuationQueue does currently
>>>     }
>>>     // do some other business logic here
>>> }
>>>
>>> Looking forward to your thoughts.
>>>
>>> Cheers,
>>> Michal
>>>
>>> -- 
>>> Signature
>>> <http://www.openbet.com/> 	Michal Borowiecki
>>> Senior Software Engineer L4
>>> 	T: 	+44 208 742 1600
>>>
>>> 	
>>> 	+44 203 249 8448
>>>
>>> 	
>>> 	
>>> 	E: 	michal.borowiecki@openbet.com
>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>
>>> 	
>>> 	OpenBet Ltd
>>>
>>> 	Chiswick Park Building 9
>>>
>>> 	566 Chiswick High Rd
>>>
>>> 	London
>>>
>>> 	W4 5XT
>>>
>>> 	UK
>>>
>>> 	
>>> <https://www.openbet.com/email_promo>
>>>
>>> This message is confidential and intended only for the addressee. If 
>>> you have received this message in error, please immediately notify 
>>> the postmaster@openbet.com <ma...@openbet.com> and 
>>> delete it from your system as well as any copies. The content of 
>>> e-mails as well as traffic data may be monitored by OpenBet for 
>>> employment and security purposes. To protect the environment please 
>>> do not print this e-mail unless necessary. OpenBet Ltd. Registered 
>>> Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 
>>> 5XT, United Kingdom. A company registered in England and Wales. 
>>> Registered no. 3134634. VAT no. GB927523612
>>>
>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If 
>> you have received this message in error, please immediately notify 
>> the postmaster@openbet.com <ma...@openbet.com> and delete 
>> it from your system as well as any copies. The content of e-mails as 
>> well as traffic data may be monitored by OpenBet for employment and 
>> security purposes. To protect the environment please do not print 
>> this e-mail unless necessary. OpenBet Ltd. Registered Office: 
>> Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT, 
>> United Kingdom. A company registered in England and Wales. Registered 
>> no. 3134634. VAT no. GB927523612
>>
>
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
>
> 	
> 	+44 203 249 8448
>
> 	
> 	
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
>
> 	
> 	OpenBet Ltd
>
> 	Chiswick Park Building 9
>
> 	566 Chiswick High Rd
>
> 	London
>
> 	W4 5XT
>
> 	UK
>
> 	
> <https://www.openbet.com/email_promo>
>
> This message is confidential and intended only for the addressee. If 
> you have received this message in error, please immediately notify the 
> postmaster@openbet.com <ma...@openbet.com> and delete it 
> from your system as well as any copies. The content of e-mails as well 
> as traffic data may be monitored by OpenBet for employment and 
> security purposes. To protect the environment please do not print this 
> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park 
> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A 
> company registered in England and Wales. Registered no. 3134634. VAT 
> no. GB927523612
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi Matthias,

Apologies, somehow I totally missed this email earlier.

Wrt ValueTransformer, I added it to the the list of deprecated methods 
(PR is up to date).

Wrt Cancellable vs Cancelable:

I'm not fluent enough to have spotted this nuance, but having googled 
for it, you are right.

On the other hand however, the precedent seems to have been set by 
java.util.concurrent.Cancellable and akka for instance followed that 
with akka.actor.Cancellable.

Given established heritage in computing context, I'd err on the side of 
consistency with prior practice.

Unless anyone has strong opinions on this matter?


Thanks,

Michal


On 04/05/17 20:43, Matthias J. Sax wrote:
> Hi,
>
> thanks for updating the KIP. Looks good to me overall.
>
> I think adding `Cancellable` (or should it be `Cancelable` to follow
> American English?) is a clean solution, in contrast to the proposed
> alternative.
>
> One minor comment: can you add `ValueTransformer#punctuate()` to the
> list of deprecated methods?
>
>
> -Matthias
>
>
>
> On 5/4/17 1:41 AM, Michal Borowiecki wrote:
>> Further in this direction I've updated the main proposal to incorporate
>> the Cancellable return type for ProcessorContext.schedule and the
>> guidance on how to implement "hybrid" punctuation with the proposed 2
>> PunctuationTypes.
>>
>> I look forward to more comments whether the Cancallable return type is
>> an agreeable solution and it's precise definition.
>>
>> I shall move all alternatives other than the main proposal into the
>> Rejected Alternatives section and if I hear any objections, I'll move
>> those back up and we'll discuss further.
>>
>>
>> Looking forward to all comments and suggestions.
>>
>>
>> Thanks,
>>
>> Michal
>>
>>
>> On 01/05/17 18:23, Michal Borowiecki wrote:
>>> Hi all,
>>>
>>> As promised, here is my take at how one could implement the previously
>>> discussed hybrid semantics using the 2 PunctuationType callbacks (one
>>> for STREAM_TIME and one for SYSTEM_TIME).
>>>
>>> However, there's a twist.
>>>
>>> Since currently calling context.schedule() adds a new
>>> PunctuationSchedule and does not overwrite the previous one, a slight
>>> change would be required:
>>>
>>> a) either that PuncuationSchedules are cancellable
>>>
>>> b) or that calling schedule() ||overwrites(cancels) the previous one
>>> with the given |PunctuationType |(but that's not how it works currently)
>>>
>>>
>>> Below is an example assuming approach a) is implemented by having
>>> schedule return Cancellable instead of void.
>>>
>>> |ProcessorContext context;|
>>> |long| |streamTimeInterval = ...;|
>>> |long| |systemTimeUpperBound = ...; ||//e.g. systemTimeUpperBound =
>>> streamTimeInterval + some tolerance|
>>> |Cancellable streamTimeSchedule;|
>>> |Cancellable systemTimeSchedule;|
>>> |long| |lastStreamTimePunctation = -||1||;|
>>> | |
>>> |public| |void| |init(ProcessorContext context){|
>>> |    ||this||.context = context;|
>>> |    ||streamTimeSchedule =
>>> context.schedule(PunctuationType.STREAM_TIME,
>>> streamTimeInterval,   ||this||::streamTimePunctuate);|
>>> |    ||systemTimeSchedule =
>>> context.schedule(PunctuationType.SYSTEM_TIME,
>>> systemTimeUpperBound, ||this||::systemTimePunctuate);   |
>>> |}|
>>> | |
>>> |public| |void| |streamTimePunctuate(||long| |streamTime){|
>>> |    ||periodicBusiness(streamTime);|
>>>   
>>> |    ||systemTimeSchedule.cancel();|
>>> |    ||systemTimeSchedule =
>>> context.schedule(PunctuationType.SYSTEM_TIME,
>>> systemTimeUpperBound, ||this||::systemTimePunctuate);|
>>> |}|
>>> | |
>>> |public| |void| |systemTimePunctuate(||long| |systemTime){|
>>> |    ||periodicBusiness(context.timestamp());|
>>>   
>>> |    ||streamTimeSchedule.cancel();|
>>> |    ||streamTimeSchedule =
>>> context.schedule(PunctuationType.STREAM_TIME,
>>> streamTimeInterval, ||this||::streamTimePunctuate);|
>>> |}|
>>> | |
>>> |public| |void| |periodicBusiness(||long| |streamTime){|
>>> |    ||// guard against streamTime == -1, easy enough.|
>>> |    ||// if you need system time instead, just use
>>> System.currentTimeMillis()|
>>> | |
>>> |    ||// do something businessy here|
>>> |}|
>>>
>>> Where Cancellable is either an interface containing just a single void
>>> cancel() method or also boolean isCancelled() like here
>>> <http://doc.akka.io/japi/akka/2.5.0/akka/actor/Cancellable.html>.
>>>
>>>
>>> Please let your opinions known whether we should proceed in this
>>> direction or leave "hybrid" considerations out of scope.
>>>
>>> Looking forward to hearing your thoughts.
>>>
>>> Thanks,
>>> Michal
>>>
>>> On 30/04/17 20:07, Michal Borowiecki wrote:
>>>> Hi Matthias,
>>>>
>>>> I'd like to start moving the discarded ideas into Rejected
>>>> Alternatives section. Before I do, I want to tidy them up, ensure
>>>> they've each been given proper treatment.
>>>>
>>>> To that end let me go back to one of your earlier comments about the
>>>> original suggestion (A) to put that to bed.
>>>>
>>>>
>>>> On 04/04/17 06:44, Matthias J. Sax wrote:
>>>>> (A) You argue, that users can still "punctuate" on event-time via
>>>>> process(), but I am not sure if this is possible. Note, that users only
>>>>> get record timestamps via context.timestamp(). Thus, users would need to
>>>>> track the time progress per partition (based on the partitions they
>>>>> obverse via context.partition(). (This alone puts a huge burden on the
>>>>> user by itself.) However, users are not notified at startup what
>>>>> partitions are assigned, and user are not notified when partitions get
>>>>> revoked. Because this information is not available, it's not possible to
>>>>> "manually advance" stream-time, and thus event-time punctuation within
>>>>> process() seems not to be possible -- or do you see a way to get it
>>>>> done? And even if, it might still be too clumsy to use.
>>>> I might have missed something but I'm guessing your worry about users
>>>> having to track time progress /per partition/ comes from the what the
>>>> stream-time does currently.
>>>> But I'm not sure that those semantics of stream-time are ideal for
>>>> users of punctuate.
>>>> That is, if stream-time punctuate didn't exist and users had to use
>>>> process(), would they actually want to use the current semantics of
>>>> stream time?
>>>>
>>>> As a reminder stream time, in all its glory, is (not exactly
>>>> actually, but when trying to be absolutely precise here I spotted
>>>> KAFKA-5144 <https://issues.apache.org/jira/browse/KAFKA-5144> so I
>>>> think this approximation suffices to illustrate the point):
>>>>
>>>> a minimum across all input partitions of (
>>>>     if(msgs never received by partition) -1;
>>>>     else {
>>>>        a non-descending-minimum of ( the per-batch minimum msg timestamp)
>>>>     }
>>>> )
>>>>
>>>> Would that really be clear enough to the users of punctuate? Do they
>>>> care for such a convoluted notion of time? I see how this can be
>>>> useful for StreamTask to pick the next partition to take a record
>>>> from but for punctuate?
>>>> If users had to implement punctuation with process(), is that what
>>>> they would have chosen as their notion of time?
>>>> I'd argue not.
>>>>
>>>> None of the processors implementing the rich windowing/join
>>>> operations in the DSL use punctuate.
>>>> Let's take the KStreamKStreamJoinProcessor as an example, in it's
>>>> process() method it simply uses context().timestamp(), which, since
>>>> it's called from process, returns simply, per javadoc:
>>>> If it is triggered while processing a record streamed from the source
>>>> processor, timestamp is defined as the timestamp of the current input
>>>> record;
>>>> So they don't use that convoluted formula for stream-time. Instead,
>>>> they only care about the timestamp of the current record. I think
>>>> that having users track just that wouldn't be that much of a burden.
>>>> I don't think they need to care about which partitions got assigned
>>>> or not. And StreamTask would still be picking records first from the
>>>> partition having the lowest timestamp to try to "synchronize" the
>>>> streams as it does now.
>>>>
>>>> What users would have to do in their Processor implementations is
>>>> somewhere along the lines of:
>>>>
>>>> long lastPunctuationTime = 0;
>>>> long interval = <some-number>; //millis
>>>>
>>>> @Override
>>>> public void process(K key, V value){
>>>>      while (ctx.timestamp() >= lastPunctuationTime + interval){
>>>>          punctuate(ctx.timestamp());
>>>>          lastPunctuationTime += interval;// I'm not sure of the merit
>>>> of this vs lastPunctuationTime = ctx.timestamp(); but that's what
>>>> PunctuationQueue does currently
>>>>      }
>>>>      // do some other business logic here
>>>> }
>>>>
>>>> Looking forward to your thoughts.
>>>>
>>>> Cheers,
>>>> Michal
>>>>
>>>> -- 
>>>> Signature
>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>> Senior Software Engineer L4
>>>> 	T: 	+44 208 742 1600
>>>>
>>>> 	
>>>> 	+44 203 249 8448
>>>>
>>>> 	
>>>> 	
>>>> 	E: 	michal.borowiecki@openbet.com
>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>
>>>> 	
>>>> 	OpenBet Ltd
>>>>
>>>> 	Chiswick Park Building 9
>>>>
>>>> 	566 Chiswick High Rd
>>>>
>>>> 	London
>>>>
>>>> 	W4 5XT
>>>>
>>>> 	UK
>>>>
>>>> 	
>>>> <https://www.openbet.com/email_promo>
>>>>
>>>> This message is confidential and intended only for the addressee. If
>>>> you have received this message in error, please immediately notify
>>>> the postmaster@openbet.com <ma...@openbet.com> and delete
>>>> it from your system as well as any copies. The content of e-mails as
>>>> well as traffic data may be monitored by OpenBet for employment and
>>>> security purposes. To protect the environment please do not print
>>>> this e-mail unless necessary. OpenBet Ltd. Registered Office:
>>>> Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
>>>> United Kingdom. A company registered in England and Wales. Registered
>>>> no. 3134634. VAT no. GB927523612
>>>>
>>> -- 
>>> Signature
>>> <http://www.openbet.com/> 	Michal Borowiecki
>>> Senior Software Engineer L4
>>> 	T: 	+44 208 742 1600
>>>
>>> 	
>>> 	+44 203 249 8448
>>>
>>> 	
>>> 	
>>> 	E: 	michal.borowiecki@openbet.com
>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>
>>> 	
>>> 	OpenBet Ltd
>>>
>>> 	Chiswick Park Building 9
>>>
>>> 	566 Chiswick High Rd
>>>
>>> 	London
>>>
>>> 	W4 5XT
>>>
>>> 	UK
>>>
>>> 	
>>> <https://www.openbet.com/email_promo>
>>>
>>> This message is confidential and intended only for the addressee. If
>>> you have received this message in error, please immediately notify the
>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>> from your system as well as any copies. The content of e-mails as well
>>> as traffic data may be monitored by OpenBet for employment and
>>> security purposes. To protect the environment please do not print this
>>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>> company registered in England and Wales. Registered no. 3134634. VAT
>>> no. GB927523612
>>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If you
>> have received this message in error, please immediately notify the
>> postmaster@openbet.com <ma...@openbet.com> and delete it
>> from your system as well as any copies. The content of e-mails as well
>> as traffic data may be monitored by OpenBet for employment and security
>> purposes. To protect the environment please do not print this e-mail
>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>> registered in England and Wales. Registered no. 3134634. VAT no.
>> GB927523612
>>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Hi,

thanks for updating the KIP. Looks good to me overall.

I think adding `Cancellable` (or should it be `Cancelable` to follow
American English?) is a clean solution, in contrast to the proposed
alternative.

One minor comment: can you add `ValueTransformer#punctuate()` to the
list of deprecated methods?


-Matthias



On 5/4/17 1:41 AM, Michal Borowiecki wrote:
> Further in this direction I've updated the main proposal to incorporate
> the Cancellable return type for ProcessorContext.schedule and the
> guidance on how to implement "hybrid" punctuation with the proposed 2
> PunctuationTypes.
> 
> I look forward to more comments whether the Cancallable return type is
> an agreeable solution and it's precise definition.
> 
> I shall move all alternatives other than the main proposal into the
> Rejected Alternatives section and if I hear any objections, I'll move
> those back up and we'll discuss further.
> 
> 
> Looking forward to all comments and suggestions.
> 
> 
> Thanks,
> 
> Michal
> 
> 
> On 01/05/17 18:23, Michal Borowiecki wrote:
>>
>> Hi all,
>>
>> As promised, here is my take at how one could implement the previously
>> discussed hybrid semantics using the 2 PunctuationType callbacks (one
>> for STREAM_TIME and one for SYSTEM_TIME).
>>
>> However, there's a twist.
>>
>> Since currently calling context.schedule() adds a new
>> PunctuationSchedule and does not overwrite the previous one, a slight
>> change would be required:
>>
>> a) either that PuncuationSchedules are cancellable
>>
>> b) or that calling schedule() ||overwrites(cancels) the previous one
>> with the given |PunctuationType |(but that's not how it works currently)
>>
>>
>> Below is an example assuming approach a) is implemented by having
>> schedule return Cancellable instead of void.
>>
>> |ProcessorContext context;|
>> |long| |streamTimeInterval = ...;|
>> |long| |systemTimeUpperBound = ...; ||//e.g. systemTimeUpperBound =
>> streamTimeInterval + some tolerance|
>> |Cancellable streamTimeSchedule;|
>> |Cancellable systemTimeSchedule;|
>> |long| |lastStreamTimePunctation = -||1||;|
>> | |
>> |public| |void| |init(ProcessorContext context){|
>> |    ||this||.context = context;|
>> |    ||streamTimeSchedule =
>> context.schedule(PunctuationType.STREAM_TIME,
>> streamTimeInterval,   ||this||::streamTimePunctuate);|
>> |    ||systemTimeSchedule =
>> context.schedule(PunctuationType.SYSTEM_TIME,
>> systemTimeUpperBound, ||this||::systemTimePunctuate);   |
>> |}|
>> | |
>> |public| |void| |streamTimePunctuate(||long| |streamTime){|
>> |    ||periodicBusiness(streamTime);|
>>  
>> |    ||systemTimeSchedule.cancel();|
>> |    ||systemTimeSchedule =
>> context.schedule(PunctuationType.SYSTEM_TIME,
>> systemTimeUpperBound, ||this||::systemTimePunctuate);|
>> |}|
>> | |
>> |public| |void| |systemTimePunctuate(||long| |systemTime){|
>> |    ||periodicBusiness(context.timestamp());|
>>  
>> |    ||streamTimeSchedule.cancel();|
>> |    ||streamTimeSchedule =
>> context.schedule(PunctuationType.STREAM_TIME,
>> streamTimeInterval, ||this||::streamTimePunctuate);|
>> |}|
>> | |
>> |public| |void| |periodicBusiness(||long| |streamTime){|
>> |    ||// guard against streamTime == -1, easy enough.|
>> |    ||// if you need system time instead, just use
>> System.currentTimeMillis()|
>> | |
>> |    ||// do something businessy here|
>> |}|
>>
>> Where Cancellable is either an interface containing just a single void
>> cancel() method or also boolean isCancelled() like here
>> <http://doc.akka.io/japi/akka/2.5.0/akka/actor/Cancellable.html>.
>>
>>
>> Please let your opinions known whether we should proceed in this
>> direction or leave "hybrid" considerations out of scope.
>>
>> Looking forward to hearing your thoughts.
>>
>> Thanks,
>> Michal
>>
>> On 30/04/17 20:07, Michal Borowiecki wrote:
>>>
>>> Hi Matthias,
>>>
>>> I'd like to start moving the discarded ideas into Rejected
>>> Alternatives section. Before I do, I want to tidy them up, ensure
>>> they've each been given proper treatment.
>>>
>>> To that end let me go back to one of your earlier comments about the
>>> original suggestion (A) to put that to bed.
>>>
>>>
>>> On 04/04/17 06:44, Matthias J. Sax wrote:
>>>> (A) You argue, that users can still "punctuate" on event-time via
>>>> process(), but I am not sure if this is possible. Note, that users only
>>>> get record timestamps via context.timestamp(). Thus, users would need to
>>>> track the time progress per partition (based on the partitions they
>>>> obverse via context.partition(). (This alone puts a huge burden on the
>>>> user by itself.) However, users are not notified at startup what
>>>> partitions are assigned, and user are not notified when partitions get
>>>> revoked. Because this information is not available, it's not possible to
>>>> "manually advance" stream-time, and thus event-time punctuation within
>>>> process() seems not to be possible -- or do you see a way to get it
>>>> done? And even if, it might still be too clumsy to use.
>>> I might have missed something but I'm guessing your worry about users
>>> having to track time progress /per partition/ comes from the what the
>>> stream-time does currently.
>>> But I'm not sure that those semantics of stream-time are ideal for
>>> users of punctuate.
>>> That is, if stream-time punctuate didn't exist and users had to use
>>> process(), would they actually want to use the current semantics of
>>> stream time?
>>>
>>> As a reminder stream time, in all its glory, is (not exactly
>>> actually, but when trying to be absolutely precise here I spotted
>>> KAFKA-5144 <https://issues.apache.org/jira/browse/KAFKA-5144> so I
>>> think this approximation suffices to illustrate the point):
>>>
>>> a minimum across all input partitions of (
>>>    if(msgs never received by partition) -1;
>>>    else {
>>>       a non-descending-minimum of ( the per-batch minimum msg timestamp)
>>>    }
>>> )
>>>
>>> Would that really be clear enough to the users of punctuate? Do they
>>> care for such a convoluted notion of time? I see how this can be
>>> useful for StreamTask to pick the next partition to take a record
>>> from but for punctuate?
>>> If users had to implement punctuation with process(), is that what
>>> they would have chosen as their notion of time?
>>> I'd argue not.
>>>
>>> None of the processors implementing the rich windowing/join
>>> operations in the DSL use punctuate.
>>> Let's take the KStreamKStreamJoinProcessor as an example, in it's
>>> process() method it simply uses context().timestamp(), which, since
>>> it's called from process, returns simply, per javadoc:
>>> If it is triggered while processing a record streamed from the source
>>> processor, timestamp is defined as the timestamp of the current input
>>> record;
>>> So they don't use that convoluted formula for stream-time. Instead,
>>> they only care about the timestamp of the current record. I think
>>> that having users track just that wouldn't be that much of a burden.
>>> I don't think they need to care about which partitions got assigned
>>> or not. And StreamTask would still be picking records first from the
>>> partition having the lowest timestamp to try to "synchronize" the
>>> streams as it does now.
>>>
>>> What users would have to do in their Processor implementations is
>>> somewhere along the lines of:
>>>
>>> long lastPunctuationTime = 0;
>>> long interval = <some-number>; //millis
>>>
>>> @Override
>>> public void process(K key, V value){
>>>     while (ctx.timestamp() >= lastPunctuationTime + interval){
>>>         punctuate(ctx.timestamp());
>>>         lastPunctuationTime += interval;// I'm not sure of the merit
>>> of this vs lastPunctuationTime = ctx.timestamp(); but that's what
>>> PunctuationQueue does currently
>>>     }
>>>     // do some other business logic here
>>> }
>>>
>>> Looking forward to your thoughts.
>>>
>>> Cheers,
>>> Michal
>>>
>>> -- 
>>> Signature
>>> <http://www.openbet.com/> 	Michal Borowiecki
>>> Senior Software Engineer L4
>>> 	T: 	+44 208 742 1600
>>>
>>> 	
>>> 	+44 203 249 8448
>>>
>>> 	
>>> 	 
>>> 	E: 	michal.borowiecki@openbet.com
>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>
>>> 	
>>> 	OpenBet Ltd
>>>
>>> 	Chiswick Park Building 9
>>>
>>> 	566 Chiswick High Rd
>>>
>>> 	London
>>>
>>> 	W4 5XT
>>>
>>> 	UK
>>>
>>> 	
>>> <https://www.openbet.com/email_promo>
>>>
>>> This message is confidential and intended only for the addressee. If
>>> you have received this message in error, please immediately notify
>>> the postmaster@openbet.com <ma...@openbet.com> and delete
>>> it from your system as well as any copies. The content of e-mails as
>>> well as traffic data may be monitored by OpenBet for employment and
>>> security purposes. To protect the environment please do not print
>>> this e-mail unless necessary. OpenBet Ltd. Registered Office:
>>> Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
>>> United Kingdom. A company registered in England and Wales. Registered
>>> no. 3134634. VAT no. GB927523612
>>>
>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	 
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If
>> you have received this message in error, please immediately notify the
>> postmaster@openbet.com <ma...@openbet.com> and delete it
>> from your system as well as any copies. The content of e-mails as well
>> as traffic data may be monitored by OpenBet for employment and
>> security purposes. To protect the environment please do not print this
>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>> company registered in England and Wales. Registered no. 3134634. VAT
>> no. GB927523612
>>
> 
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
> 
> 	
> 	+44 203 249 8448
> 
> 	
> 	 
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
> 
> 	
> 	OpenBet Ltd
> 
> 	Chiswick Park Building 9
> 
> 	566 Chiswick High Rd
> 
> 	London
> 
> 	W4 5XT
> 
> 	UK
> 
> 	
> <https://www.openbet.com/email_promo>
> 
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com <ma...@openbet.com> and delete it
> from your system as well as any copies. The content of e-mails as well
> as traffic data may be monitored by OpenBet for employment and security
> purposes. To protect the environment please do not print this e-mail
> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> registered in England and Wales. Registered no. 3134634. VAT no.
> GB927523612
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Further in this direction I've updated the main proposal to incorporate 
the Cancellable return type for ProcessorContext.schedule and the 
guidance on how to implement "hybrid" punctuation with the proposed 2 
PunctuationTypes.

I look forward to more comments whether the Cancallable return type is 
an agreeable solution and it's precise definition.

I shall move all alternatives other than the main proposal into the 
Rejected Alternatives section and if I hear any objections, I'll move 
those back up and we'll discuss further.


Looking forward to all comments and suggestions.


Thanks,

Michal


On 01/05/17 18:23, Michal Borowiecki wrote:
>
> Hi all,
>
> As promised, here is my take at how one could implement the previously 
> discussed hybrid semantics using the 2 PunctuationType callbacks (one 
> for STREAM_TIME and one for SYSTEM_TIME).
>
> However, there's a twist.
>
> Since currently calling context.schedule() adds a new 
> PunctuationSchedule and does not overwrite the previous one, a slight 
> change would be required:
>
> a) either that PuncuationSchedules are cancellable
>
> b) or that calling schedule() ||overwrites(cancels) the previous one 
> with the given |PunctuationType |(but that's not how it works currently)
>
>
> Below is an example assuming approach a) is implemented by having 
> schedule return Cancellable instead of void.
>
> |ProcessorContext context;|
> |long||streamTimeInterval = ...;|
> |long||systemTimeUpperBound = ...;||//e.g. systemTimeUpperBound = 
> streamTimeInterval + some tolerance|
> |Cancellable streamTimeSchedule;|
> |Cancellable systemTimeSchedule;|
> |long||lastStreamTimePunctation = -||1||;|
> ||
> |public||void||init(ProcessorContext context){|
> |||this||.context = context;|
> |||streamTimeSchedule = context.schedule(PunctuationType.STREAM_TIME, 
> streamTimeInterval, ||this||::streamTimePunctuate);|
> |||systemTimeSchedule = context.schedule(PunctuationType.SYSTEM_TIME, 
> systemTimeUpperBound,||this||::systemTimePunctuate); |
> |}|
> ||
> |public||void||streamTimePunctuate(||long||streamTime){|
> |||periodicBusiness(streamTime);|
> |||systemTimeSchedule.cancel();|
> |||systemTimeSchedule = context.schedule(PunctuationType.SYSTEM_TIME, 
> systemTimeUpperBound,||this||::systemTimePunctuate);|
> |}|
> ||
> |public||void||systemTimePunctuate(||long||systemTime){|
> |||periodicBusiness(context.timestamp());|
> |||streamTimeSchedule.cancel();|
> |||streamTimeSchedule = context.schedule(PunctuationType.STREAM_TIME, 
> streamTimeInterval,||this||::streamTimePunctuate);|
> |}|
> ||
> |public||void||periodicBusiness(||long||streamTime){|
> |||// guard against streamTime == -1, easy enough.|
> |||// if you need system time instead, just use 
> System.currentTimeMillis()|
> ||
> |||// do something businessy here|
> |}|
>
> Where Cancellable is either an interface containing just a single void 
> cancel() method or also boolean isCancelled() likehere 
> <http://doc.akka.io/japi/akka/2.5.0/akka/actor/Cancellable.html>.
>
>
> Please let your opinions known whether we should proceed in this 
> direction or leave "hybrid" considerations out of scope.
>
> Looking forward to hearing your thoughts.
>
> Thanks,
> Michal
>
> On 30/04/17 20:07, Michal Borowiecki wrote:
>>
>> Hi Matthias,
>>
>> I'd like to start moving the discarded ideas into Rejected 
>> Alternatives section. Before I do, I want to tidy them up, ensure 
>> they've each been given proper treatment.
>>
>> To that end let me go back to one of your earlier comments about the 
>> original suggestion (A) to put that to bed.
>>
>>
>> On 04/04/17 06:44, Matthias J. Sax wrote:
>>> (A) You argue, that users can still "punctuate" on event-time via
>>> process(), but I am not sure if this is possible. Note, that users only
>>> get record timestamps via context.timestamp(). Thus, users would need to
>>> track the time progress per partition (based on the partitions they
>>> obverse via context.partition(). (This alone puts a huge burden on the
>>> user by itself.) However, users are not notified at startup what
>>> partitions are assigned, and user are not notified when partitions get
>>> revoked. Because this information is not available, it's not possible to
>>> "manually advance" stream-time, and thus event-time punctuation within
>>> process() seems not to be possible -- or do you see a way to get it
>>> done? And even if, it might still be too clumsy to use.
>> I might have missed something but I'm guessing your worry about users 
>> having to track time progress /per partition/ comes from the what the 
>> stream-time does currently.
>> But I'm not sure that those semantics of stream-time are ideal for 
>> users of punctuate.
>> That is, if stream-time punctuate didn't exist and users had to use 
>> process(), would they actually want to use the current semantics of 
>> stream time?
>>
>> As a reminder stream time, in all its glory, is (not exactly 
>> actually, but when trying to be absolutely precise here I spotted 
>> KAFKA-5144 <https://issues.apache.org/jira/browse/KAFKA-5144> so I 
>> think this approximation suffices to illustrate the point):
>>
>> a minimum across all input partitions of (
>>    if(msgs never received by partition) -1;
>>    else {
>>       a non-descending-minimum of ( the per-batch minimum msg timestamp)
>>    }
>> )
>>
>> Would that really be clear enough to the users of punctuate? Do they 
>> care for such a convoluted notion of time? I see how this can be 
>> useful for StreamTask to pick the next partition to take a record 
>> from but for punctuate?
>> If users had to implement punctuation with process(), is that what 
>> they would have chosen as their notion of time?
>> I'd argue not.
>>
>> None of the processors implementing the rich windowing/join 
>> operations in the DSL use punctuate.
>> Let's take the KStreamKStreamJoinProcessor as an example, in it's 
>> process() method it simply uses context().timestamp(), which, since 
>> it's called from process, returns simply, per javadoc:
>> If it is triggered while processing a record streamed from the source 
>> processor, timestamp is defined as the timestamp of the current input 
>> record;
>> So they don't use that convoluted formula for stream-time. Instead, 
>> they only care about the timestamp of the current record. I think 
>> that having users track just that wouldn't be that much of a burden. 
>> I don't think they need to care about which partitions got assigned 
>> or not. And StreamTask would still be picking records first from the 
>> partition having the lowest timestamp to try to "synchronize" the 
>> streams as it does now.
>>
>> What users would have to do in their Processor implementations is 
>> somewhere along the lines of:
>>
>> long lastPunctuationTime = 0;
>> long interval = <some-number>; //millis
>>
>> @Override
>> public void process(K key, V value){
>>     while (ctx.timestamp() >= lastPunctuationTime + interval){
>>         punctuate(ctx.timestamp());
>>         lastPunctuationTime += interval;// I'm not sure of the merit 
>> of this vs lastPunctuationTime = ctx.timestamp(); but that's what 
>> PunctuationQueue does currently
>>     }
>>     // do some other business logic here
>> }
>>
>> Looking forward to your thoughts.
>>
>> Cheers,
>> Michal
>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If 
>> you have received this message in error, please immediately notify 
>> the postmaster@openbet.com <ma...@openbet.com> and delete 
>> it from your system as well as any copies. The content of e-mails as 
>> well as traffic data may be monitored by OpenBet for employment and 
>> security purposes. To protect the environment please do not print 
>> this e-mail unless necessary. OpenBet Ltd. Registered Office: 
>> Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT, 
>> United Kingdom. A company registered in England and Wales. Registered 
>> no. 3134634. VAT no. GB927523612
>>
>
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
>
> 	
> 	+44 203 249 8448
>
> 	
> 	
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
>
> 	
> 	OpenBet Ltd
>
> 	Chiswick Park Building 9
>
> 	566 Chiswick High Rd
>
> 	London
>
> 	W4 5XT
>
> 	UK
>
> 	
> <https://www.openbet.com/email_promo>
>
> This message is confidential and intended only for the addressee. If 
> you have received this message in error, please immediately notify the 
> postmaster@openbet.com <ma...@openbet.com> and delete it 
> from your system as well as any copies. The content of e-mails as well 
> as traffic data may be monitored by OpenBet for employment and 
> security purposes. To protect the environment please do not print this 
> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park 
> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A 
> company registered in England and Wales. Registered no. 3134634. VAT 
> no. GB927523612
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi all,

As promised, here is my take at how one could implement the previously 
discussed hybrid semantics using the 2 PunctuationType callbacks (one 
for STREAM_TIME and one for SYSTEM_TIME).

However, there's a twist.

Since currently calling context.schedule() adds a new 
PunctuationSchedule and does not overwrite the previous one, a slight 
change would be required:

a) either that PuncuationSchedules are cancellable

b) or that calling schedule() ||overwrites(cancels) the previous one 
with the given |PunctuationType |(but that's not how it works currently)


Below is an example assuming approach a) is implemented by having 
schedule return Cancellable instead of void.

|ProcessorContext context;|
|long||streamTimeInterval = ...;|
|long||systemTimeUpperBound = ...;||//e.g. systemTimeUpperBound = 
streamTimeInterval + some tolerance|
|Cancellable streamTimeSchedule;|
|Cancellable systemTimeSchedule;|
|long||lastStreamTimePunctation = -||1||;|
||
|public||void||init(ProcessorContext context){|
|||this||.context = context;|
|||streamTimeSchedule = context.schedule(PunctuationType.STREAM_TIME, 
streamTimeInterval, ||this||::streamTimePunctuate);|
|||systemTimeSchedule = context.schedule(PunctuationType.SYSTEM_TIME, 
systemTimeUpperBound,||this||::systemTimePunctuate); |
|}|
||
|public||void||streamTimePunctuate(||long||streamTime){|
|||periodicBusiness(streamTime);|
|||systemTimeSchedule.cancel();|
|||systemTimeSchedule = context.schedule(PunctuationType.SYSTEM_TIME, 
systemTimeUpperBound,||this||::systemTimePunctuate);|
|}|
||
|public||void||systemTimePunctuate(||long||systemTime){|
|||periodicBusiness(context.timestamp());|
|||streamTimeSchedule.cancel();|
|||streamTimeSchedule = context.schedule(PunctuationType.STREAM_TIME, 
streamTimeInterval,||this||::streamTimePunctuate);|
|}|
||
|public||void||periodicBusiness(||long||streamTime){|
|||// guard against streamTime == -1, easy enough.|
|||// if you need system time instead, just use System.currentTimeMillis()|
||
|||// do something businessy here|
|}|

Where Cancellable is either an interface containing just a single void 
cancel() method or also boolean isCancelled() likehere 
<http://doc.akka.io/japi/akka/2.5.0/akka/actor/Cancellable.html>.


Please let your opinions known whether we should proceed in this 
direction or leave "hybrid" considerations out of scope.

Looking forward to hearing your thoughts.

Thanks,
Michal

On 30/04/17 20:07, Michal Borowiecki wrote:
>
> Hi Matthias,
>
> I'd like to start moving the discarded ideas into Rejected 
> Alternatives section. Before I do, I want to tidy them up, ensure 
> they've each been given proper treatment.
>
> To that end let me go back to one of your earlier comments about the 
> original suggestion (A) to put that to bed.
>
>
> On 04/04/17 06:44, Matthias J. Sax wrote:
>> (A) You argue, that users can still "punctuate" on event-time via
>> process(), but I am not sure if this is possible. Note, that users only
>> get record timestamps via context.timestamp(). Thus, users would need to
>> track the time progress per partition (based on the partitions they
>> obverse via context.partition(). (This alone puts a huge burden on the
>> user by itself.) However, users are not notified at startup what
>> partitions are assigned, and user are not notified when partitions get
>> revoked. Because this information is not available, it's not possible to
>> "manually advance" stream-time, and thus event-time punctuation within
>> process() seems not to be possible -- or do you see a way to get it
>> done? And even if, it might still be too clumsy to use.
> I might have missed something but I'm guessing your worry about users 
> having to track time progress /per partition/ comes from the what the 
> stream-time does currently.
> But I'm not sure that those semantics of stream-time are ideal for 
> users of punctuate.
> That is, if stream-time punctuate didn't exist and users had to use 
> process(), would they actually want to use the current semantics of 
> stream time?
>
> As a reminder stream time, in all its glory, is (not exactly actually, 
> but when trying to be absolutely precise here I spotted KAFKA-5144 
> <https://issues.apache.org/jira/browse/KAFKA-5144> so I think this 
> approximation suffices to illustrate the point):
>
> a minimum across all input partitions of (
>    if(msgs never received by partition) -1;
>    else {
>       a non-descending-minimum of ( the per-batch minimum msg timestamp)
>    }
> )
>
> Would that really be clear enough to the users of punctuate? Do they 
> care for such a convoluted notion of time? I see how this can be 
> useful for StreamTask to pick the next partition to take a record from 
> but for punctuate?
> If users had to implement punctuation with process(), is that what 
> they would have chosen as their notion of time?
> I'd argue not.
>
> None of the processors implementing the rich windowing/join operations 
> in the DSL use punctuate.
> Let's take the KStreamKStreamJoinProcessor as an example, in it's 
> process() method it simply uses context().timestamp(), which, since 
> it's called from process, returns simply, per javadoc:
> If it is triggered while processing a record streamed from the source 
> processor, timestamp is defined as the timestamp of the current input 
> record;
> So they don't use that convoluted formula for stream-time. Instead, 
> they only care about the timestamp of the current record. I think that 
> having users track just that wouldn't be that much of a burden. I 
> don't think they need to care about which partitions got assigned or 
> not. And StreamTask would still be picking records first from the 
> partition having the lowest timestamp to try to "synchronize" the 
> streams as it does now.
>
> What users would have to do in their Processor implementations is 
> somewhere along the lines of:
>
> long lastPunctuationTime = 0;
> long interval = <some-number>; //millis
>
> @Override
> public void process(K key, V value){
>     while (ctx.timestamp() >= lastPunctuationTime + interval){
>         punctuate(ctx.timestamp());
>         lastPunctuationTime += interval;// I'm not sure of the merit 
> of this vs lastPunctuationTime = ctx.timestamp(); but that's what 
> PunctuationQueue does currently
>     }
>     // do some other business logic here
> }
>
> Looking forward to your thoughts.
>
> Cheers,
> Michal
>
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
>
> 	
> 	+44 203 249 8448
>
> 	
> 	
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
>
> 	
> 	OpenBet Ltd
>
> 	Chiswick Park Building 9
>
> 	566 Chiswick High Rd
>
> 	London
>
> 	W4 5XT
>
> 	UK
>
> 	
> <https://www.openbet.com/email_promo>
>
> This message is confidential and intended only for the addressee. If 
> you have received this message in error, please immediately notify the 
> postmaster@openbet.com <ma...@openbet.com> and delete it 
> from your system as well as any copies. The content of e-mails as well 
> as traffic data may be monitored by OpenBet for employment and 
> security purposes. To protect the environment please do not print this 
> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park 
> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A 
> company registered in England and Wales. Registered no. 3134634. VAT 
> no. GB927523612
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi Matthias,

I'd like to start moving the discarded ideas into Rejected Alternatives 
section. Before I do, I want to tidy them up, ensure they've each been 
given proper treatment.

To that end let me go back to one of your earlier comments about the 
original suggestion (A) to put that to bed.


On 04/04/17 06:44, Matthias J. Sax wrote:
> (A) You argue, that users can still "punctuate" on event-time via
> process(), but I am not sure if this is possible. Note, that users only
> get record timestamps via context.timestamp(). Thus, users would need to
> track the time progress per partition (based on the partitions they
> obverse via context.partition(). (This alone puts a huge burden on the
> user by itself.) However, users are not notified at startup what
> partitions are assigned, and user are not notified when partitions get
> revoked. Because this information is not available, it's not possible to
> "manually advance" stream-time, and thus event-time punctuation within
> process() seems not to be possible -- or do you see a way to get it
> done? And even if, it might still be too clumsy to use.
I might have missed something but I'm guessing your worry about users 
having to track time progress /per partition/ comes from the what the 
stream-time does currently.
But I'm not sure that those semantics of stream-time are ideal for users 
of punctuate.
That is, if stream-time punctuate didn't exist and users had to use 
process(), would they actually want to use the current semantics of 
stream time?

As a reminder stream time, in all its glory, is (not exactly actually, 
but when trying to be absolutely precise here I spotted KAFKA-5144 
<https://issues.apache.org/jira/browse/KAFKA-5144> so I think this 
approximation suffices to illustrate the point):

a minimum across all input partitions of (
    if(msgs never received by partition) -1;
    else {
       a non-descending-minimum of ( the per-batch minimum msg timestamp)
    }
)

Would that really be clear enough to the users of punctuate? Do they 
care for such a convoluted notion of time? I see how this can be useful 
for StreamTask to pick the next partition to take a record from but for 
punctuate?
If users had to implement punctuation with process(), is that what they 
would have chosen as their notion of time?
I'd argue not.

None of the processors implementing the rich windowing/join operations 
in the DSL use punctuate.
Let's take the KStreamKStreamJoinProcessor as an example, in it's 
process() method it simply uses context().timestamp(), which, since it's 
called from process, returns simply, per javadoc:

If it is triggered while processing a record streamed from the source 
processor, timestamp is defined as the timestamp of the current input 
record;

So they don't use that convoluted formula for stream-time. Instead, they 
only care about the timestamp of the current record. I think that having 
users track just that wouldn't be that much of a burden. I don't think 
they need to care about which partitions got assigned or not. And 
StreamTask would still be picking records first from the partition 
having the lowest timestamp to try to "synchronize" the streams as it 
does now.

What users would have to do in their Processor implementations is 
somewhere along the lines of:

long lastPunctuationTime = 0;
long interval = <some-number>; //millis

@Override
public void process(K key, V value){
     while (ctx.timestamp() >= lastPunctuationTime + interval){
         punctuate(ctx.timestamp());
         lastPunctuationTime += interval;// I'm not sure of the merit of 
this vs lastPunctuationTime = ctx.timestamp(); but that's what 
PunctuationQueue does currently
     }
     // do some other business logic here
}

Looking forward to your thoughts.

Cheers,
Michal

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi Arun,

Thanks for your reply.


Indeed, the flaw I described is not there. Semantically, at least, it 
will behave correctly.

However, I see a different flaw now, an operational one:

If it's triggered by system time first then and next schedule is set to 
(0+interval) (or precisely it would be (-1 + interval) as in the absence 
of records on any of the partitions, the stream time == -1).

When msgs start coming in, the stream time timestamps are going to be, 
say, around 1493565370568.

So now, a stream-time caused punctuation is going to be triggered and 
the next schedule (since the special guard case of 
PunctuationSchedule.timestamp==0 is no longer met) is going to be set 
for PunctuationSchedule.timestamp + interval, which is 
(-1+interval+interval) (NOTE: *not* current stream time + interval, 
since PunctuationSchedule.next only uses the currTimestamp argument in 
the guard for the special case mentioned above ). Since this is still in 
the past wrt stream time, the loop is going to retrieve the freshly 
created PunctuationSchedule again and increment by interval for the the 
next schedule, again and again.

Assuming an interval of 10s for simplicity that means it's going to 
trigger punctuate 1493565370568/10000 ~= 150 million times before the 
schedule catches up to the current stream time.

I think that's a pain.

Please let me know if I got it wrong somewhere.


As to:

> My thought as to why minimum was chosen can be illustrated via a simple
> use-case. Say we are consuming data from topic A event{ts, valueA} and
> topic B  event{ts, valueB}, and output to topic C event{ts, valueC}. If
> we have to output the valueC = valueA + valueB for matching ts values,
> then we need to punctuate to happen after all values for a specified ts
> has arrived. (of course we are skipping the case of missing events,
> late/out of order events etc).
Having read other email threads I now believe this was chosen to pick 
the next record to process from the partition that has the smallest 
timestamp. But I don't think that's necessarily the best approach. e.g. 
see my comment about non-determinism here: 
https://issues.apache.org/jira/browse/KAFKA-3514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15990176#comment-15990176

Your example seems reasonable too, but currently the join and windowing 
operations in the DSL don't seem to be using punctuate at all. So, I'd 
argue that for the scenario you describe it's best not to use punctuate 
but either the DSL or, if you need to go down to the Processor API, 
implement it in the same fashion as those underlying Processors for 
joins and windowing are implemented. They do handle late arriving 
records with some tolerance I believe, controlled by the window 
retention time. In general, I can't yet see why stream time couldn't be 
the maximum of all partition timestamps (for a more regular advance) and 
all but the latest (as per their timestamp) records be equally 
considered late-arriving and dealt with as late-arriving records are 
dealt with now.

There may still be a lot I don't understand in this area and it's out of 
scope of this KIP anyway. But I think separately to this KIP, we should 
keep the discussion going around the semantics of stream time.

Cheers,
Michal


On 28/04/17 06:54, Matthias J. Sax wrote:
> Thanks for sharing Arun!
>
> -Matthias
>
> On 4/26/17 5:59 AM, Arun Mathew wrote:
>> Hi All,
>>
>>   
>>
>> Apologies for the delayed response.
>>
>> As requested by Tianji, I am sharing our implementation of hybrid punctuate.
>>
>>   
>>
>> It is a rather small patch, a hack for our use-case I would say. I
>> suggest you all to have a look.
>>
>> You can view/(download) the diff from
>> https://github.com/apache/kafka/compare/0.10.1.1...arunmathew88:hybrid-punctuate
>> (.diff)
>>
>>   
>>
>> Now I will try to address some of matters below
>>
>>   
>>
>>> In the current stream-time punctuation scheme, the first
>> PunctuationSchedule is set to 0 + interval (what else, since we haven't
>> seen any records yet), but when triggered, it passes the current stream
>> time as timestamp argument and uses that to schedule the next
>> punctuation. It all makes sense.
>>> Arun, have you implemented any special handling for this case in your
>> system?
>>
>> In our implementation, we never send the system time as argument to
>> punctuate, as it will be semantically incorrect to mix it. We use system
>> time only to check if punctuate interval has elapsed in the wallclock.
>> So in the situation of the 0 + interval as fist punctuation schedule,
>> our hybrid punctuate implementation will call punctuate with
>> (0+interval) as the argument, and (0 + interval  + interval) as the next
>> punctuation schedule. Hence this flaw is not there.
>>
>>   
>>
>> However as I had mentioned in my original description mail (under the
>> Cons section), and as pointed out by Matthias
>>
>>   
>>
>>> (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
>> implicit assumption that even-time progresses at the same "speed" as
>> system-time during regular processing. This implies the assumption that
>> a slower progress in stream-time indicates the absence of input events
>>
>>   
>>
>> Though this can be tweaked to detect a slow stream time progression to
>> block hybrid punctuation, it won\u2019t look very clean.
>>
>>   
>>
>> @Michal, regarding 2b
>>
>>> but I can't believe this to be the only reason behind this choice as
>> minimum is not the only function to guarantee the group partition time
>> never going back. Using the largest or the average among partitions'
>> timestamp would also guaranteed the group timestamp not going back as
>> timestamp never goes back for any individual partition.
>>
>> So why was minimum chosen? Is it depended on by window semantics
>> somewhere or anything else?
>>
>>   
>>
>> My thought as to why minimum was chosen can be illustrated via a simple
>> use-case. Say we are consuming data from topic A event{ts, valueA} and
>> topic B  event{ts, valueB}, and output to topic C event{ts, valueC}. If
>> we have to output the valueC = valueA + valueB for matching ts values,
>> then we need to punctuate to happen after all values for a specified ts
>> has arrived. (of course we are skipping the case of missing events,
>> late/out of order events etc).
>>
>>   
>>
>>   
>>
>>> I agree that expressing the intended hybrid semantics is harder if we
>> offer only #StreamTime and #SystemTime punctuation. However, I also
>>
>> believe that the hybrid approach is a "false friend" with regard to
>>
>> reasoning about the semantics (it indicates that it more easy as it is
>>
>> in reality). Therefore, we might be better off to not offer the hybrid
>>
>> approach and make it clear to a developed, that it is hard to mix
>>
>> #StreamTime and #SystemTime in a semantically sound way.
>>
>>   
>>
>> I agree with Matthias, that hybrid approach might not be a clean
>> approach, it has its own assumptions, In particular I don\u2019t like that
>> the approach will have to virtually force push the stream time, for a
>> low event rate partition in the partition group, to trigger a punctuate.
>>   Nevertheless, I believe it is quite handy in a few use-cases.
>>
>>   
>>
>> I totally agree with the manthra of keeping things simple and straight
>> forward at the platform level and let the developer build things on top
>> of it. So let us think how to modify the punctuate semantics so that the
>> user can implement desired effect, stream time /system time /hybrid
>> punctuate at the developer level. I had to patch the Kafka Streams
>> because I wasn\u2019t able to implement it using the levers and switches
>> provided by the KS library. I had originally just wanted a pure system
>> time based punctuate, like the cron case discussed before.
>>
>>   
>>
>>   
>>
>>   
>>
>> It was a long read, since my last mail, I will go through all
>> conversations again and update the Use-Cases document appropriately.
>>
>>   
>>
>> --
>>
>> With Regards
>>
>>   
>>
>> Arun Mathew
>>
>> Yahoo! JAPAN Corporation
>>
>>   
>>
>>   
>>
>> *From: *Michal Borowiecki <mi...@openbet.com>
>> *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>> *Date: *Tuesday, April 25, 2017 at 07:11
>> *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>> *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
>>
>>   
>>
>> Sorry, my original point was lost or I didn't make it clear enough.
>>
>> I don't challenge the need for a system-time interval upper bound, one
>> we don't have now. On the contrary my own use-case depends on it.
>> However, the hybrid does provide a system-time interval upper bound,
>> even in the absence of records, it only fails to provide a lower bound.
>>
>> As to heartbeats, IMO that wouldn't need a lower bound on the heartbeat
>> interval, an upper bound should suffice.
>>
>>   
>>
>> Nevertheless, I think I found a flaw in the hybrid proposal, as follows:
>>
>> In the current stream-time punctuation scheme, the first
>> PunctuationSchedule is set to 0 + interval (what else, since we haven't
>> seen any records yet), but when triggered, it passes the current stream
>> time as timestamp argument and uses that to schedule the next
>> punctuation. It all makes sense.
>>
>> In the hybrid this would only work if the first punctuation is triggered
>> by stream time advance but could have unintended consequences if the
>> first triggering is by system time.
>>
>> Imagine this scenario: my apps are down and I have loads of records to
>> process (perhaps I reset offsets to reprocess data).
>>
>> If I start my streams app but records don't start coming in immediately
>> (perhaps there is another app upstream that I have yet to start), then
>> the first punctuation will be triggered by system time. It won't have
>> any stream-time to use so, the only timestamp it can use is the
>> system-time timestamp, both as an argument to punctuate and for the next
>> schedule.
>>
>> But if then messages start flowing and have event time way in the past
>> (assuming event/ingestion time extractor), then stream time will be
>> behind system-time until the stream catches up (which may be never if I
>> just want to run in historical data as an experiment/validation).
>> However, punctuate (having been triggered first by system time) will
>> keep on recurring at regular intervals and always passing in the system
>> time (or to be precise: the system time of previous punctuation +
>> interval). That is surely not what is desired or expected, given the
>> steady inflow of records.
>>
>> Arun, have you implemented any special handling for this case in your
>> system?
>>
>> As mentioned before the hybrid punctuation scheme is not easy to reason
>> about, I can't deny that.
>>
>>   
>>
>> Happy to stick with the 2 PunctuationTypes proposal (stream time and
>> system time). I'll try to write up how users can implement hybrid-like
>> semantics themselves on top of that to address those demanding use-cases
>> mentioned on the wiki.
>>
>> Maybe it won't end up too complex after all, but as per the mantra, it's
>> enough that it's made possible ;-)
>>
>> Thanks,
>>
>> Michal
>>
>>   
>>
>> On 24/04/17 18:22, Matthias J. Sax wrote:
>>
>>              Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?
>>
>>      If you go with stream-time and don't have any input records for all
>>
>>      partitions, punctuate would not be called at all, and thus your
>>
>>      dashboard would "freeze".
>>
>>       
>>
>>              I thought about cron-type things, but aren't they better triggered by an
>>
>>              external scheduler (they're more flexible anyway), which then feeds
>>
>>              "commands" into the topology?
>>
>>      I guess it depends what kind of periodic action you want to trigger. The
>>
>>      "cron job" was just an analogy. Maybe it's just a heartbeat to some
>>
>>      other service, that signals that your Streams app is still running.
>>
>>       
>>
>>       
>>
>>      -Matthias
>>
>>       
>>
>>       
>>
>>      On 4/24/17 10:02 AM, Michal Borowiecki wrote:
>>
>>          Thanks!
>>
>>           
>>
>>          Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?
>>
>>           
>>
>>          Unless too frequent messages on replay could overpower it?
>>
>>           
>>
>>           
>>
>>          I thought about cron-type things, but aren't they better triggered by an
>>
>>          external scheduler (they're more flexible anyway), which then feeds
>>
>>          "commands" into the topology?
>>
>>           
>>
>>          Just my 2c.
>>
>>           
>>
>>          Cheers,
>>
>>           
>>
>>          Michal
>>
>>           
>>
>>           
>>
>>          On 24/04/17 17:57, Matthias J. Sax wrote:
>>
>>              A simple example would be some dashboard app, that needs to get
>>
>>              "current" status in regular time intervals (ie, and real-time app).
>>
>>               
>>
>>              Or something like a "scheduler" -- think "cron job" application.
>>
>>               
>>
>>               
>>
>>              -Matthias
>>
>>               
>>
>>              On 4/24/17 2:23 AM, Michal Borowiecki wrote:
>>
>>                  Hi Matthias,
>>
>>                   
>>
>>                  I agree it's difficult to reason about the hybrid approach, I certainly
>>
>>                  found it hard and I'm totally on board with the mantra.
>>
>>                   
>>
>>                  I'd be happy to limit the scope of this KIP to add system-time
>>
>>                  punctuation semantics (in addition to existing stream-time semantics)
>>
>>                  and leave more complex schemes for users to implement on top of that.
>>
>>                   
>>
>>                  Further additional PunctuationTypes, could then be added by future KIPs,
>>
>>                  possibly including the hybrid approach once it has been given more thought.
>>
>>                   
>>
>>                      There are real-time applications, that want to get
>>
>>                      callbacks in regular system-time intervals (completely independent from
>>
>>                      stream-time).
>>
>>                  Can you please describe what they are, so that I can put them on the
>>
>>                  wiki for later reference?
>>
>>                   
>>
>>                  Thanks,
>>
>>                   
>>
>>                  Michal
>>
>>                   
>>
>>                   
>>
>>                  On 23/04/17 21:27, Matthias J. Sax wrote:
>>
>>                      Hi,
>>
>>                       
>>
>>                      I do like Damian's API proposal about the punctuation callback function.
>>
>>                       
>>
>>                      I also did reread the KIP and thought about the semantics we want to
>>
>>                      provide.
>>
>>                       
>>
>>                          Given the above, I don't see a reason any more for a separate system-time based punctuation.
>>
>>                      I disagree here. There are real-time applications, that want to get
>>
>>                      callbacks in regular system-time intervals (completely independent from
>>
>>                      stream-time). Thus we should allow this -- if we really follow the
>>
>>                      "hybrid" approach, this could be configured with stream-time interval
>>
>>                      infinite and delay whatever system-time punctuation interval you want to
>>
>>                      have. However, I would like to add a proper API for this and do this
>>
>>                      configuration under the hood (that would allow one implementation within
>>
>>                      all kind of branching for different cases).
>>
>>                       
>>
>>                      Thus, we definitely should have PunctutionType#StreamTime and
>>
>>                      #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
>>
>>                      a fan of your latest API proposal.
>>
>>                       
>>
>>                       
>>
>>                      About the hybrid approach in general. On the one hand I like it, on the
>>
>>                      other hand, it seems to be rather (1) complicated (not necessarily from
>>
>>                      an implementation point of view, but for people to understand it) and
>>
>>                      (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
>>
>>                       
>>
>>                          It may appear complicated at first but I do think these semantics will
>>
>>                          still be more understandable to users than having 2 separate punctuation
>>
>>                          schedules/callbacks with different PunctuationTypes.
>>
>>                      This statement only holds if you apply strong assumptions that I don't
>>
>>                      believe hold in general -- see (2) for details -- and I think it is
>>
>>                      harder than you assume to reason about the hybrid approach in general.
>>
>>                      IMHO, the hybrid approach is a "false friend" that seems to be easy to
>>
>>                      reason about...
>>
>>                       
>>
>>                       
>>
>>                      (1) Streams always embraced "easy to use" and we should really be
>>
>>                      careful to keep it this way. On the other hand, as we are talking about
>>
>>                      changes to PAPI, it won't affect DSL users (DSL does not use punctuation
>>
>>                      at all at the moment), and thus, the "easy to use" mantra might not be
>>
>>                      affected, while it will allow advanced users to express more complex stuff.
>>
>>                       
>>
>>                      I like the mantra: "make simple thing easy and complex things possible".
>>
>>                       
>>
>>                      (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
>>
>>                      implicit assumption that even-time progresses at the same "speed" as
>>
>>                      system-time during regular processing. This implies the assumption that
>>
>>                      a slower progress in stream-time indicates the absence of input events
>>
>>                      (and that later arriving input events will have a larger event-time with
>>
>>                      high probability). Even if this might be true for some use cases, I
>>
>>                      doubt it holds in general. Assume that you get a spike in traffic and
>>
>>                      for some reason stream-time does advance slowly because you have more
>>
>>                      records to process. This might trigger a system-time based punctuation
>>
>>                      call even if this seems not to be intended. I strongly believe that it
>>
>>                      is not easy to reason about the semantics of the hybrid approach (even
>>
>>                      if the intentional semantics would be super useful -- but I doubt that
>>
>>                      we get want we ask for).
>>
>>                       
>>
>>                      Thus, I also believe that one might need different "configuration"
>>
>>                      values for the hybrid approach if you run the same code for different
>>
>>                      scenarios: regular processing, re-processing, catching up scenario. And
>>
>>                      as the term "configuration" implies, we might be better off to not mix
>>
>>                      configuration with business logic that is expressed via code.
>>
>>                       
>>
>>                       
>>
>>                      One more comment: I also don't think that the hybrid approach is
>>
>>                      deterministic as claimed in the use-case subpage. I understand the
>>
>>                      reasoning and agree, that it is deterministic if certain assumptions
>>
>>                      hold -- compare above -- and if configured correctly. But strictly
>>
>>                      speaking it's not because there is a dependency on system-time (and
>>
>>                      IMHO, if system-time is involved it cannot be deterministic by definition).
>>
>>                       
>>
>>                       
>>
>>                          I see how in theory this could be implemented on top of the 2 punctuate
>>
>>                          callbacks with the 2 different PunctuationTypes (one stream-time based,
>>
>>                          the other system-time based) but it would be a much more complicated
>>
>>                          scheme and I don't want to suggest that.
>>
>>                      I agree that expressing the intended hybrid semantics is harder if we
>>
>>                      offer only #StreamTime and #SystemTime punctuation. However, I also
>>
>>                      believe that the hybrid approach is a "false friend" with regard to
>>
>>                      reasoning about the semantics (it indicates that it more easy as it is
>>
>>                      in reality). Therefore, we might be better off to not offer the hybrid
>>
>>                      approach and make it clear to a developed, that it is hard to mix
>>
>>                      #StreamTime and #SystemTime in a semantically sound way.
>>
>>                       
>>
>>                       
>>
>>                      Looking forward to your feedback. :)
>>
>>                       
>>
>>                      -Matthias
>>
>>                       
>>
>>                       
>>
>>                       
>>
>>                       
>>
>>                      On 4/22/17 11:43 AM, Michal Borowiecki wrote:
>>
>>                          Hi all,
>>
>>                           
>>
>>                          Looking for feedback on the functional interface approach Damian
>>
>>                          proposed. What do people think?
>>
>>                           
>>
>>                          Further on the semantics of triggering punctuate though:
>>
>>                           
>>
>>                          I ran through the 2 use cases that Arun had kindly put on the wiki
>>
>>                          (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
>>
>>                          in my head and on a whiteboard and I can't find a better solution than
>>
>>                          the "hybrid" approach he had proposed.
>>
>>                           
>>
>>                          I see how in theory this could be implemented on top of the 2 punctuate
>>
>>                          callbacks with the 2 different PunctuationTypes (one stream-time based,
>>
>>                          the other system-time based) but it would be a much more complicated
>>
>>                          scheme and I don't want to suggest that.
>>
>>                           
>>
>>                          However, to add to the hybrid algorithm proposed, I suggest one
>>
>>                          parameter to that: a tolerance period, expressed in milliseconds
>>
>>                          system-time, after which the punctuation will be invoked in case the
>>
>>                          stream-time advance hasn't triggered it within the requested interval
>>
>>                          since the last invocation of punctuate (as recorded in system-time)
>>
>>                           
>>
>>                          This would allow a user-defined tolerance for late arriving events. The
>>
>>                          trade off would be left for the user to decide: regular punctuation in
>>
>>                          the case of absence of events vs allowing for records arriving late or
>>
>>                          some build-up due to processing not catching up with the event rate.
>>
>>                          In the one extreme, this tolerance could be set to infinity, turning
>>
>>                          hybrid into simply stream-time based punctuate, like we have now. In the
>>
>>                          other extreme, the tolerance could be set to 0, resulting in a
>>
>>                          system-time upper bound on the effective punctuation interval.
>>
>>                           
>>
>>                          Given the above, I don't see a reason any more for a separate
>>
>>                          system-time based punctuation. The "hybrid" approach with 0ms tolerance
>>
>>                          would under normal operation trigger at regular intervals wrt the
>>
>>                          system-time, except in cases of re-play/catch-up, where the stream time
>>
>>                          advances faster than system time. In these cases punctuate would happen
>>
>>                          more often than the specified interval wrt system time. However, the
>>
>>                          use-cases that need system-time punctuations (that I've seen at least)
>>
>>                          really only have a need for an upper bound on punctuation delay but
>>
>>                          don't need a lower bound.
>>
>>                           
>>
>>                          To that effect I'd propose the api to be as follows, on ProcessorContext:
>>
>>                           
>>
>>                          schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
>>
>>                           
>>
>>                          schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
>>
>>                           
>>
>>                          Punctuation is triggered when either:
>>
>>                          - the stream time advances past the (stream time of the previous
>>
>>                          punctuation) + interval;
>>
>>                          - or (iff the toleranceInterval is set) when the system time advances
>>
>>                          past the (system time of the previous punctuation) + interval +
>>
>>                          toleranceInterval
>>
>>                           
>>
>>                          In either case:
>>
>>                          - we trigger punctuate passing as the argument the stream time at which
>>
>>                          the current punctuation was meant to happen
>>
>>                          - next punctuate is scheduled at (stream time at which the current
>>
>>                          punctuation was meant to happen) + interval
>>
>>                           
>>
>>                          It may appear complicated at first but I do think these semantics will
>>
>>                          still be more understandable to users than having 2 separate punctuation
>>
>>                          schedules/callbacks with different PunctuationTypes.
>>
>>                           
>>
>>                           
>>
>>                           
>>
>>                          PS. Having re-read this, maybe the following alternative would be easier
>>
>>                          to understand (WDYT?):
>>
>>                           
>>
>>                          schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
>>
>>                           
>>
>>                          schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
>>
>>                           
>>
>>                          Punctuation is triggered when either:
>>
>>                          - the stream time advances past the (stream time of the previous
>>
>>                          punctuation) + streamTimeInterval;
>>
>>                          - or (iff systemTimeUpperBound is set) when the system time advances
>>
>>                          past the (system time of the previous punctuation) + systemTimeUpperBound
>>
>>                           
>>
>>                          Awaiting comments.
>>
>>                           
>>
>>                          Thanks,
>>
>>                          Michal
>>
>>                           
>>
>>                          On 21/04/17 16:56, Michal Borowiecki wrote:
>>
>>                              Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>>
>>                              in favour of something that doesn't return a record. Not a problem though.
>>
>>                               
>>
>>                               
>>
>>                              On 21/04/17 16:32, Damian Guy wrote:
>>
>>                                  Thanks Michal,
>>
>>                                  I agree Transformer.punctuate should also be void, but we can deprecate
>>
>>                                  that too in favor of the new interface.
>>
>>                                   
>>
>>                                  Thanks for the javadoc PR!
>>
>>                                   
>>
>>                                  Cheers,
>>
>>                                  Damian
>>
>>                                   
>>
>>                                  On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>
>>                                  michal.borowiecki@openbet.com
>>                                  <ma...@openbet.com>> wrote:
>>
>>                                   
>>
>>                                      Yes, that looks better to me.
>>
>>                                       
>>
>>                                      Note that punctuate on Transformer is currently returning a record, but I
>>
>>                                      think it's ok to have all output records be sent via
>>
>>                                      ProcessorContext.forward, which has to be used anyway if you want to send
>>
>>                                      multiple records from one invocation of punctuate.
>>
>>                                       
>>
>>                                      This way it's consistent between Processor and Transformer.
>>
>>                                       
>>
>>                                       
>>
>>                                      BTW, looking at this I found a glitch in the javadoc and put a comment
>>
>>                                      there:
>>
>>                                       
>>
>>                                      https://github.com/apache/kafka/pull/2413/files#r112634612
>>
>>                                       
>>
>>                                      and PR: https://github.com/apache/kafka/pull/2884
>>
>>                                       
>>
>>                                      Cheers,
>>
>>                                       
>>
>>                                      Michal
>>
>>                                      On 20/04/17 18:55, Damian Guy wrote:
>>
>>                                       
>>
>>                                      Hi Michal,
>>
>>                                       
>>
>>                                      Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>
>>                                      the API.
>>
>>                                      1. deprecate the punctuate method on Processor
>>
>>                                      2. create a new Functional Interface just for Punctuation, something like:
>>
>>                                      interface Punctuator {
>>
>>                                          void punctuate(long timestamp)
>>
>>                                      }
>>
>>                                      3. add a new schedule function to ProcessorContext: schedule(long
>>
>>                                      interval, PunctuationType type, Punctuator callback)
>>
>>                                      4. deprecate the existing schedule function
>>
>>                                       
>>
>>                                      Thoughts?
>>
>>                                       
>>
>>                                      Thanks,
>>
>>                                      Damian
>>
>>                                       
>>
>>                                      On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>
>>                                      michal.borowiecki@openbet.com
>>                                      <ma...@openbet.com>> wrote:
>>
>>                                       
>>
>>                                          Hi Thomas,
>>
>>                                           
>>
>>                                          I would say our use cases fall in the same category as yours.
>>
>>                                           
>>
>>                                          1) One is expiry of old records, it's virtually identical to yours.
>>
>>                                           
>>
>>                                          2) Second one is somewhat more convoluted but boils down to the same type
>>
>>                                          of design:
>>
>>                                           
>>
>>                                          Incoming messages carry a number of fields, including a timestamp.
>>
>>                                           
>>
>>                                          Outgoing messages contain derived fields, one of them (X) is depended on
>>
>>                                          by the timestamp input field (Y) and some other input field (Z).
>>
>>                                           
>>
>>                                          Since the output field X is derived in some non-trivial way, we don't
>>
>>                                          want to force the logic onto downstream apps. Instead we want to calculate
>>
>>                                          it in the Kafka Streams app, which means we re-calculate X as soon as the
>>
>>                                          timestamp in Y is reached (wall clock time) and send a message if it
>>
>>                                          changed (I say "if" because the derived field (X) is also conditional on
>>
>>                                          another input field Z).
>>
>>                                           
>>
>>                                          So we have kv stores with the records and an additional kv store with
>>
>>                                          timestamp->id mapping which act like an index where we periodically do a
>>
>>                                          ranged query.
>>
>>                                           
>>
>>                                          Initially we naively tried doing it in punctuate which of course didn't
>>
>>                                          work when there were no regular msgs on the input topic.
>>
>>                                          Since this was before 0.10.1 and state stores weren't query-able from
>>
>>                                          outside we created a "ticker" that produced msgs once per second onto
>>
>>                                          another topic and fed it into the same topology to trigger punctuate.
>>
>>                                          This didn't work either, which was much more surprising to us at the
>>
>>                                          time, because it was not obvious at all that punctuate is only triggered if
>>
>>                                          *all* input partitions receive messages regularly.
>>
>>                                          In the end we had to break this into 2 separate Kafka Streams. Main
>>
>>                                          transformer doesn't use punctuate but sends values of timestamp field Y and
>>
>>                                          the id to a "scheduler" topic where also the periodic ticks are sent. This
>>
>>                                          is consumed by the second topology and is its only input topic. There's a
>>
>>                                          transformer on that topic which populates and updates the time-based
>>
>>                                          indexes and polls them from punctuate. If the time in the timestamp
>>
>>                                          elapsed, the record id is sent to the main transformer, which
>>
>>                                          updates/deletes the record from the main kv store and forwards the
>>
>>                                          transformed record to the output topic.
>>
>>                                           
>>
>>                                          To me this setup feels horrendously complicated for what it does.
>>
>>                                           
>>
>>                                          We could incrementally improve on this since 0.10.1 to poll the
>>
>>                                          timestamp->id "index" stores from some code outside the KafkaStreams
>>
>>                                          topology so that at least we wouldn't need the extra topic for "ticks".
>>
>>                                          However, the ticks don't feel so hacky when you realise they give you
>>
>>                                          some hypothetical benefits in predictability. You can reprocess the
>>
>>                                          messages in a reproducible manner, since the topologies use event-time,
>>
>>                                          just that the event time is simply the wall-clock time fed into a topic by
>>
>>                                          the ticks. (NB in our use case we haven't yet found a need for this kind of
>>
>>                                          reprocessing).
>>
>>                                          To make that work though, we would have to have the stream time advance
>>
>>                                          based on the presence of msgs on the "tick" topic, regardless of the
>>
>>                                          presence of messages on the other input topic.
>>
>>                                           
>>
>>                                          Same as in the expiry use case, both the wall-clock triggered punctuate
>>
>>                                          and the hybrid would work to simplify this a lot.
>>
>>                                           
>>
>>                                          3) Finally, I have a 3rd use case in the making but I'm still looking if
>>
>>                                          we can achieve it using session windows instead. I'll keep you posted if we
>>
>>                                          have to go with punctuate there too.
>>
>>                                           
>>
>>                                          Thanks,
>>
>>                                          Michal
>>
>>                                           
>>
>>                                           
>>
>>                                          On 11/04/17 20:52, Thomas Becker wrote:
>>
>>                                           
>>
>>                                          Here's an example that we currently have.  We have a streams processor
>>
>>                                          that does a transform from one topic into another. One of the fields in
>>
>>                                          the source topic record is an expiration time, and one of the functions
>>
>>                                          of the processor is to ensure that expired records get deleted promptly
>>
>>                                          after that time passes (typically days or weeks after the message was
>>
>>                                          originally produced). To do that, the processor keeps a state store of
>>
>>                                          keys and expiration times, iterates that store in punctuate(), and
>>
>>                                          emits delete (null) records for expired items. This needs to happen at
>>
>>                                          some minimum interval regardless of the incoming message rate of the
>>
>>                                          source topic.
>>
>>                                           
>>
>>                                          In this scenario, the expiration of records is the primary function of
>>
>>                                          punctuate, and therefore the key requirement is that the wall-clock
>>
>>                                          measured time between punctuate calls have some upper-bound. So a pure
>>
>>                                          wall-clock based schedule would be fine for our needs. But the proposed
>>
>>                                          "hybrid" system would also be acceptable if that satisfies a broader
>>
>>                                          range of use-cases.
>>
>>                                           
>>
>>                                          On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>
>>                                           
>>
>>                                          I apologize for the longer email below.  To my defense, it started
>>
>>                                          out much
>>
>>                                          shorter. :-)  Also, to be super-clear, I am intentionally playing
>>
>>                                          devil's
>>
>>                                          advocate for a number of arguments brought forth in order to help
>>
>>                                          improve
>>
>>                                          this KIP -- I am not implying I necessarily disagree with the
>>
>>                                          arguments.
>>
>>                                           
>>
>>                                          That aside, here are some further thoughts.
>>
>>                                           
>>
>>                                          First, there are (at least?) two categories for actions/behavior you
>>
>>                                          invoke
>>
>>                                          via punctuate():
>>
>>                                           
>>
>>                                          1. For internal housekeeping of your Processor or Transformer (e.g.,
>>
>>                                          to
>>
>>                                          periodically commit to a custom store, to do metrics/logging).  Here,
>>
>>                                          the
>>
>>                                          impact of punctuate is typically not observable by other processing
>>
>>                                          nodes
>>
>>                                          in the topology.
>>
>>                                          2. For controlling the emit frequency of downstream records.  Here,
>>
>>                                          the
>>
>>                                          punctuate is all about being observable by downstream processing
>>
>>                                          nodes.
>>
>>                                           
>>
>>                                          A few releases back, we introduced record caches (DSL) and state
>>
>>                                          store
>>
>>                                          caches (Processor API) in KIP-63.  Here, we addressed a concern
>>
>>                                          relating to
>>
>>                                          (2) where some users needed to control -- here: limit -- the
>>
>>                                          downstream
>>
>>                                          output rate of Kafka Streams because the downstream systems/apps
>>
>>                                          would not
>>
>>                                          be able to keep up with the upstream output rate (Kafka scalability >
>>
>>                                          their
>>
>>                                          scalability).  The argument for KIP-63, which notably did not
>>
>>                                          introduce a
>>
>>                                          "trigger" API, was that such an interaction with downstream systems
>>
>>                                          is an
>>
>>                                          operational concern;  it should not impact the processing *logic* of
>>
>>                                          your
>>
>>                                          application, and thus we didn't want to complicate the Kafka Streams
>>
>>                                          API,
>>
>>                                          especially not the declarative DSL, with such operational concerns.
>>
>>                                           
>>
>>                                          This KIP's discussion on `punctuate()` takes us back in time (<--
>>
>>                                          sorry, I
>>
>>                                          couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>
>>                                          observing that our conversation is moving more and more into the
>>
>>                                          direction
>>
>>                                          of explicit "triggers" because, so far, I have seen only motivations
>>
>>                                          for
>>
>>                                          use cases in category (2), but none yet for (1)?  For example, some
>>
>>                                          comments voiced here are about sth like "IF stream-time didn't
>>
>>                                          trigger
>>
>>                                          punctuate, THEN trigger punctuate based on processing-time".  Do we
>>
>>                                          want
>>
>>                                          this, and if so, for which use cases and benefits?  Also, on a
>>
>>                                          related
>>
>>                                          note, whatever we are discussing here will impact state store caches
>>
>>                                          (Processor API) and perhaps also impact record caches (DSL), thus we
>>
>>                                          should
>>
>>                                          clarify any such impact here.
>>
>>                                           
>>
>>                                          Switching topics slightly.
>>
>>                                           
>>
>>                                          Jay wrote:
>>
>>                                           
>>
>>                                          One thing I've always found super important for this kind of design
>>
>>                                          work
>>
>>                                          is to do a really good job of cataloging the landscape of use cases
>>
>>                                          and
>>
>>                                          how prevalent each one is.
>>
>>                                           
>>
>>                                          +1 to this, as others have already said.
>>
>>                                           
>>
>>                                          Here, let me highlight -- just in case -- that when we talked about
>>
>>                                          windowing use cases in the recent emails, the Processor API (where
>>
>>                                          `punctuate` resides) does not have any notion of windowing at
>>
>>                                          all.  If you
>>
>>                                          want to do windowing *in the Processor API*, you must do so manually
>>
>>                                          in
>>
>>                                          combination with window stores.  For this reason I'd suggest to
>>
>>                                          discuss use
>>
>>                                          cases not just in general, but also in view of how you'd do so in the
>>
>>                                          Processor API vs. in the DSL.  Right now, changing/improving
>>
>>                                          `punctuate`
>>
>>                                          does not impact the DSL at all, unless we add new functionality to
>>
>>                                          it.
>>
>>                                           
>>
>>                                          Jay wrote in his strawman example:
>>
>>                                           
>>
>>                                          You aggregate click and impression data for a reddit like site.
>>
>>                                          Every ten
>>
>>                                          minutes you want to output a ranked list of the top 10 articles
>>
>>                                          ranked by
>>
>>                                          clicks/impressions for each geographical area. I want to be able
>>
>>                                          run this
>>
>>                                          in steady state as well as rerun to regenerate results (or catch up
>>
>>                                          if it
>>
>>                                          crashes).
>>
>>                                           
>>
>>                                          This is a good example for more than the obvious reason:  In KIP-63,
>>
>>                                          we
>>
>>                                          argued that the reason for saying "every ten minutes" above is not
>>
>>                                          necessarily about because you want to output data *exactly* after ten
>>
>>                                          minutes, but that you want to perform an aggregation based on 10-
>>
>>                                          minute
>>
>>                                          windows of input data; i.e., the point is about specifying the input
>>
>>                                          for
>>
>>                                          your aggregation, not or less about when the results of the
>>
>>                                          aggregation
>>
>>                                          should be send downstream.  To take an extreme example, you could
>>
>>                                          disable
>>
>>                                          record caches and let your app output a downstream update for every
>>
>>                                          incoming input record.  If the last input record was from at minute 7
>>
>>                                          of 10
>>
>>                                          (for a 10-min window), then what your app would output at minute 10
>>
>>                                          would
>>
>>                                          be identical to what it had already emitted at minute 7 earlier
>>
>>                                          anyways.
>>
>>                                          This is particularly true when we take late-arriving data into
>>
>>                                          account:  if
>>
>>                                          a late record arrived at minute 13, your app would (by default) send
>>
>>                                          a new
>>
>>                                          update downstream, even though the "original" 10 minutes have already
>>
>>                                          passed.
>>
>>                                           
>>
>>                                          Jay wrote...:
>>
>>                                           
>>
>>                                          There are a couple of tricky things that seem to make this hard
>>
>>                                          with
>>
>>                                           
>>
>>                                          either
>>
>>                                           
>>
>>                                          of the options proposed:
>>
>>                                          1. If I emit this data using event time I have the problem
>>
>>                                          described where
>>
>>                                          a geographical region with no new clicks or impressions will fail
>>
>>                                          to
>>
>>                                           
>>
>>                                          output
>>
>>                                           
>>
>>                                          results.
>>
>>                                           
>>
>>                                          ...and Arun Mathew wrote:
>>
>>                                           
>>
>>                                           
>>
>>                                          We window by the event time, but trigger punctuate in <punctuate
>>
>>                                          interval>
>>
>>                                          duration of system time, in the absence of an event crossing the
>>
>>                                          punctuate
>>
>>                                          event time.
>>
>>                                           
>>
>>                                          So, given what I wrote above about the status quo and what you can
>>
>>                                          already
>>
>>                                          do with it, is the concern that the state store cache doesn't give
>>
>>                                          you
>>
>>                                          *direct* control over "forcing an output after no later than X
>>
>>                                          seconds [of
>>
>>                                          processing-time]" but only indirect control through a cache
>>
>>                                          size?  (Note
>>
>>                                          that I am not dismissing the claims why this might be helpful.)
>>
>>                                           
>>
>>                                          Arun Mathew wrote:
>>
>>                                           
>>
>>                                          We are using Kafka Stream for our Audit Trail, where we need to
>>
>>                                          output the
>>
>>                                          event counts on each topic on each cluster aggregated over a 1
>>
>>                                          minute
>>
>>                                          window. We have to use event time to be able to cross check the
>>
>>                                          counts.
>>
>>                                           
>>
>>                                          But
>>
>>                                           
>>
>>                                          we need to trigger punctuate [aggregate event pushes] by system
>>
>>                                          time in
>>
>>                                           
>>
>>                                          the
>>
>>                                           
>>
>>                                          absence of events. Otherwise the event counts for unexpired windows
>>
>>                                          would
>>
>>                                          be 0 which is bad.
>>
>>                                           
>>
>>                                          Isn't the latter -- "count would be 0" -- the problem between the
>>
>>                                          absence
>>
>>                                          of output vs. an output of 0, similar to the use of `Option[T]` in
>>
>>                                          Scala
>>
>>                                          and the difference between `None` and `Some(0)`?  That is, isn't the
>>
>>                                          root
>>
>>                                          cause that the downstream system interprets the absence of output in
>>
>>                                          a
>>
>>                                          particular way ("No output after 1 minute = I consider the output to
>>
>>                                          be
>>
>>                                          0.")?  Arguably, you could also adapt the downstream system (if
>>
>>                                          possible)
>>
>>                                          to correctly handle the difference between absence of output vs.
>>
>>                                          output of
>>
>>                                          0.  I am not implying that we shouldn't care about such a use case,
>>
>>                                          but
>>
>>                                          want to understand the motivation better. :-)
>>
>>                                           
>>
>>                                          Also, to add some perspective, in some related discussions we talked
>>
>>                                          about
>>
>>                                          how a Kafka Streams application should not worry or not be coupled
>>
>>                                          unnecessarily with such interpretation specifics in a downstream
>>
>>                                          system's
>>
>>                                          behavior.  After all, tomorrow your app's output might be consumed by
>>
>>                                          more
>>
>>                                          than just this one downstream system.  Arguably, Kafka Connect rather
>>
>>                                          than
>>
>>                                          Kafka Streams might be the best tool to link the universes of Kafka
>>
>>                                          and
>>
>>                                          downstream systems, including helping to reconcile the differences in
>>
>>                                          how
>>
>>                                          these systems interpret changes, updates, late-arriving data,
>>
>>                                          etc.  Kafka
>>
>>                                          Connect would allow you to decouple the Kafka Streams app's logical
>>
>>                                          processing from the specifics of downstream systems, thanks to
>>
>>                                          specific
>>
>>                                          sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>
>>                                          this
>>
>>                                          decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>
>>                                          but
>>
>>                                          it's currently awkward to use Connect for this", this might be a
>>
>>                                          problem we
>>
>>                                          can solve, too.)
>>
>>                                           
>>
>>                                          Switching topics slightly again.
>>
>>                                           
>>
>>                                          Thomas wrote:
>>
>>                                           
>>
>>                                          I'm not entirely convinced that a separate callback (option C)
>>
>>                                          is that messy (it could just be a default method with an empty
>>
>>                                          implementation), but if we wanted a single API to handle both
>>
>>                                          cases,
>>
>>                                          how about something like the following?
>>
>>                                           
>>
>>                                          enum Time {
>>
>>                                             STREAM,
>>
>>                                             CLOCK
>>
>>                                          }
>>
>>                                           
>>
>>                                          Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>
>>                                          then
>>
>>                                          whatever the user is doing inside this method is a black box to Kafka
>>
>>                                          Streams (similar to how we have no idea what the user does inside a
>>
>>                                          `foreach` -- if the function passed to `foreach` writes to external
>>
>>                                          systems, then Kafka Streams is totally unaware of the fact).  We
>>
>>                                          won't
>>
>>                                          know, for example, if the stream-time action has a smaller "trigger"
>>
>>                                          frequency than the processing-time action.  Or, we won't know whether
>>
>>                                          the
>>
>>                                          user custom-codes a "not later than" trigger logic ("Do X every 1-
>>
>>                                          minute of
>>
>>                                          stream-time or 1-minute of processing-time, whichever comes
>>
>>                                          first").  That
>>
>>                                          said, I am not certain yet whether we would need such knowledge
>>
>>                                          because,
>>
>>                                          when using the Processor API, most of the work and decisions must be
>>
>>                                          done
>>
>>                                          by the user anyways.  It would matter though if the concept of
>>
>>                                          "triggers"
>>
>>                                          were to bubble up into the DSL because in the DSL the management of
>>
>>                                          windowing, window stores, etc. must be done automatically by Kafka
>>
>>                                          Streams.
>>
>>                                           
>>
>>                                          [In any case, btw, we have the corner case where the user configured
>>
>>                                          the
>>
>>                                          stream-time to be processing-time (e.g. via wall-clock timestamp
>>
>>                                          extractor), at which point both punctuate variants are based on the
>>
>>                                          same
>>
>>                                          time semantics / timeline.]
>>
>>                                           
>>
>>                                          Again, I apologize for the wall of text.  Congratulations if you made
>>
>>                                          it
>>
>>                                          this far. :-)
>>
>>                                           
>>
>>                                          More than happy to hear your thoughts!
>>
>>                                          Michael
>>
>>                                           
>>
>>                                          On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com>
>>                                          <ma...@gmail.com> <ar...@gmail.com>
>>                                          <ma...@gmail.com>
>>
>>                                          wrote:
>>
>>                                           
>>
>>                                           
>>
>>                                          Thanks Matthias.
>>
>>                                          Sure, will correct it right away.
>>
>>                                           
>>
>>                                          On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io>
>>                                          <ma...@confluent.io> <ma...@confluent.io>
>>                                          <ma...@confluent.io>
>>
>>                                          wrote:
>>
>>                                           
>>
>>                                          Thanks for preparing this page!
>>
>>                                           
>>
>>                                          About terminology:
>>
>>                                           
>>
>>                                          You introduce the term "event time" -- but we should call this
>>
>>                                          "stream
>>
>>                                          time" -- "stream time" is whatever TimestampExtractor returns and
>>
>>                                          this
>>
>>                                          could be event time, ingestion time, or processing/wall-clock time.
>>
>>                                           
>>
>>                                          Does this make sense to you?
>>
>>                                           
>>
>>                                           
>>
>>                                           
>>
>>                                          -Matthias
>>
>>                                           
>>
>>                                           
>>
>>                                          On 4/10/17 4:58 AM, Arun Mathew wrote:
>>
>>                                           
>>
>>                                          Thanks Ewen.
>>
>>                                           
>>
>>                                          @Michal, @all, I have created a child page to start the Use Cases
>>
>>                                           
>>
>>                                          discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>
>>                                          Punctuate+Use+Cases]. Please go through it and give your comments.
>>
>>                                           
>>
>>                                           
>>
>>                                          @Tianji, Sorry for the delay. I am trying to make the patch
>>
>>                                          public.
>>
>>                                           
>>
>>                                          --
>>
>>                                          Arun Mathew
>>
>>                                           
>>
>>                                          On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io>
>>                                          <ma...@confluent.io> <ew...@confluent.io>
>>                                          <ma...@confluent.io>
>>
>>                                          wrote:
>>
>>                                           
>>
>>                                              Arun,
>>
>>                                           
>>
>>                                              I've given you permission to edit the wiki. Let me know if
>>
>>                                          you run
>>
>>                                           
>>
>>                                          into any
>>
>>                                           
>>
>>                                              issues.
>>
>>                                           
>>
>>                                              -Ewen
>>
>>                                           
>>
>>                                              On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp>
>>                                          <ma...@yahoo-corp.jp> <am...@yahoo-corp.jp>
>>                                          <ma...@yahoo-corp.jp>
>>
>>                                           
>>
>>                                          wrote:
>>
>>                                           
>>
>>                                           
>>
>>                                              > Thanks Michal. I don\u2019t have the access yet [arunmathew88].
>>
>>                                          Should I
>>
>>                                           
>>
>>                                          be
>>
>>                                           
>>
>>                                              > sending a separate mail for this?
>>
>>                                              >
>>
>>                                              > I thought one of the person following this thread would be
>>
>>                                          able to
>>
>>                                           
>>
>>                                          give me
>>
>>                                           
>>
>>                                              > access.
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > *From: *Michal Borowiecki <mi...@openbet.com>
>>                                          <ma...@openbet.com> <mi...@openbet.com>
>>                                          <ma...@openbet.com>
>>
>>                                              > *Reply-To: *"dev@kafka.apache.org"
>>                                          <ma...@kafka.apache.org> <de...@kafka.apache.org>
>>                                          <ma...@kafka.apache.org> <de...@kafka.apache.org>
>>                                          <ma...@kafka.apache.org> <de...@kafka.apache.org>
>>                                          <ma...@kafka.apache.org>
>>
>>                                              > *Date: *Friday, April 7, 2017 at 17:16
>>
>>                                              > *To: *"dev@kafka.apache.org"
>>                                          <ma...@kafka.apache.org> <de...@kafka.apache.org>
>>                                          <ma...@kafka.apache.org> <de...@kafka.apache.org>
>>                                          <ma...@kafka.apache.org> <de...@kafka.apache.org>
>>                                          <ma...@kafka.apache.org>
>>
>>                                              > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>
>>                                          semantics
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > Hi Arun,
>>
>>                                              >
>>
>>                                              > I was thinking along the same lines as you, listing the use
>>
>>                                          cases
>>
>>                                           
>>
>>                                          on the
>>
>>                                           
>>
>>                                              > wiki, but didn't find time to get around doing that yet.
>>
>>                                              > Don't mind if you do it if you have access now.
>>
>>                                              > I was thinking it would be nice if, once we have the use
>>
>>                                          cases
>>
>>                                           
>>
>>                                          listed,
>>
>>                                           
>>
>>                                              > people could use likes to up-vote the use cases similar to
>>
>>                                          what
>>
>>                                           
>>
>>                                          they're
>>
>>                                           
>>
>>                                              > working on.
>>
>>                                              >
>>
>>                                              > I should have a bit more time to action this in the next
>>
>>                                          few days,
>>
>>                                           
>>
>>                                          but
>>
>>                                           
>>
>>                                              > happy for you to do it if you can beat me to it ;-)
>>
>>                                              >
>>
>>                                              > Cheers,
>>
>>                                              > Michal
>>
>>                                              >
>>
>>                                              > On 07/04/17 04:39, Arun Mathew wrote:
>>
>>                                              >
>>
>>                                              > Sure, Thanks Matthias. My id is [arunmathew88].
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > Of course. I was thinking of a subpage where people can
>>
>>                                           
>>
>>                                          collaborate.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > Will do as per Michael\u2019s suggestion.
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > Regards,
>>
>>                                              >
>>
>>                                              > Arun Mathew
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io>
>>                                          <ma...@confluent.io> <ma...@confluent.io>
>>                                          <ma...@confluent.io>
>>
>>                                          <
>>
>>                                           
>>
>>                                          matthias@confluent.io
>>                                          <ma...@confluent.io>> wrote:
>>
>>                                           
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >     Please share your Wiki-ID and a committer can give you
>>
>>                                          write
>>
>>                                           
>>
>>                                          access.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >     Btw: as you did not initiate the KIP, you should not
>>
>>                                          change the
>>
>>                                           
>>
>>                                          KIP
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     without the permission of the original author -- in
>>
>>                                          this case
>>
>>                                           
>>
>>                                          Michael.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >     So you might also just share your thought over the
>>
>>                                          mailing list
>>
>>                                           
>>
>>                                          and
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     Michael can update the KIP page. Or, as an alternative,
>>
>>                                          just
>>
>>                                           
>>
>>                                          create a
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     subpage for the KIP page.
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >     @Michael: WDYT?
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >     -Matthias
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>
>>                                              >
>>
>>                                              >     > Hi Jay,
>>
>>                                              >
>>
>>                                              >     >           Thanks for the advise, I would like to list
>>
>>                                          down
>>
>>                                           
>>
>>                                          the use cases as
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > per your suggestion. But it seems I don't have write
>>
>>                                           
>>
>>                                          permission to the
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > Apache Kafka Confluent Space. Whom shall I request
>>
>>                                          for it?
>>
>>                                              >
>>
>>                                              >     >
>>
>>                                              >
>>
>>                                              >     > Regarding your last question. We are using a patch in
>>
>>                                          our
>>
>>                                           
>>
>>                                          production system
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > which does exactly this.
>>
>>                                              >
>>
>>                                              >     > We window by the event time, but trigger punctuate in
>>
>>                                           
>>
>>                                          <punctuate interval>
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > duration of system time, in the absence of an event
>>
>>                                          crossing
>>
>>                                           
>>
>>                                          the punctuate
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > event time.
>>
>>                                              >
>>
>>                                              >     >
>>
>>                                              >
>>
>>                                              >     > We are using Kafka Stream for our Audit Trail, where
>>
>>                                          we need
>>
>>                                           
>>
>>                                          to output the
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > event counts on each topic on each cluster aggregated
>>
>>                                          over a
>>
>>                                           
>>
>>                                          1 minute
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > window. We have to use event time to be able to cross
>>
>>                                          check
>>
>>                                           
>>
>>                                          the counts. But
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > we need to trigger punctuate [aggregate event pushes]
>>
>>                                          by
>>
>>                                           
>>
>>                                          system time in the
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > absence of events. Otherwise the event counts for
>>
>>                                          unexpired
>>
>>                                           
>>
>>                                          windows would
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > be 0 which is bad.
>>
>>                                              >
>>
>>                                              >     >
>>
>>                                              >
>>
>>                                              >     > "Maybe a hybrid solution works: I window by event
>>
>>                                          time but
>>
>>                                           
>>
>>                                          trigger results
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > by system time for windows that have updated? Not
>>
>>                                          really sure
>>
>>                                           
>>
>>                                          the details
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > of making that work. Does that work? Are there
>>
>>                                          concrete
>>
>>                                           
>>
>>                                          examples where you
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > actually want the current behavior?"
>>
>>                                              >
>>
>>                                              >     >
>>
>>                                              >
>>
>>                                              >     > --
>>
>>                                              >
>>
>>                                              >     > With Regards,
>>
>>                                              >
>>
>>                                              >     >
>>
>>                                              >
>>
>>                                              >     > Arun Mathew
>>
>>                                              >
>>
>>                                              >     > Yahoo! JAPAN Corporation
>>
>>                                              >
>>
>>                                              >     >
>>
>>                                              >
>>
>>                                              >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>
>>                                           
>>
>>                                          skyahead@gmail.com
>>                                          <ma...@gmail.com>
>>                                          <ma...@gmail.com> <sk...@gmail.com>
>>                                          <ma...@gmail.com> wrote:
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >
>>
>>                                              >
>>
>>                                              >     >> Hi Jay,
>>
>>                                              >
>>
>>                                              >     >>
>>
>>                                              >
>>
>>                                              >     >> The hybrid solution is exactly what I expect and
>>
>>                                          need for
>>
>>                                           
>>
>>                                          our use cases
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> when dealing with telecom data.
>>
>>                                              >
>>
>>                                              >     >>
>>
>>                                              >
>>
>>                                              >     >> Thanks
>>
>>                                              >
>>
>>                                              >     >> Tianji
>>
>>                                              >
>>
>>                                              >     >>
>>
>>                                              >
>>
>>                                              >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>
>>                                           
>>
>>                                          jay@confluent.io
>>                                          <ma...@confluent.io>
>>                                          <ma...@confluent.io> <ja...@confluent.io>
>>                                          <ma...@confluent.io> wrote:
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>
>>
>>                                              >
>>
>>                                              >     >>> Hey guys,
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>> One thing I've always found super important for
>>
>>                                          this kind
>>
>>                                           
>>
>>                                          of design work
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> is
>>
>>                                              >
>>
>>                                              >     >>> to do a really good job of cataloging the landscape
>>
>>                                          of use
>>
>>                                           
>>
>>                                          cases and how
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> prevalent each one is. By that I mean not just
>>
>>                                          listing lots
>>
>>                                           
>>
>>                                          of uses, but
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> also grouping them into categories that
>>
>>                                          functionally need
>>
>>                                           
>>
>>                                          the same thing.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> In the absence of this it is very hard to reason
>>
>>                                          about
>>
>>                                           
>>
>>                                          design proposals.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> From the proposals so far I think we have a lot of
>>
>>                                           
>>
>>                                          discussion around
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> possible apis, but less around what the user needs
>>
>>                                          for
>>
>>                                           
>>
>>                                          different use
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> cases
>>
>>                                              >
>>
>>                                              >     >>> and how they would implement that using the api.
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>> Here is an example:
>>
>>                                              >
>>
>>                                              >     >>> You aggregate click and impression data for a
>>
>>                                          reddit like
>>
>>                                           
>>
>>                                          site. Every ten
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> minutes you want to output a ranked list of the top
>>
>>                                          10
>>
>>                                           
>>
>>                                          articles ranked by
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> clicks/impressions for each geographical area. I
>>
>>                                          want to be
>>
>>                                           
>>
>>                                          able run this
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> in steady state as well as rerun to regenerate
>>
>>                                          results (or
>>
>>                                           
>>
>>                                          catch up if it
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> crashes).
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>> There are a couple of tricky things that seem to
>>
>>                                          make this
>>
>>                                           
>>
>>                                          hard with
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> either
>>
>>                                              >
>>
>>                                              >     >>> of the options proposed:
>>
>>                                              >
>>
>>                                              >     >>> 1. If I emit this data using event time I have the
>>
>>                                          problem
>>
>>                                           
>>
>>                                          described
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> where
>>
>>                                              >
>>
>>                                              >     >>> a geographical region with no new clicks or
>>
>>                                          impressions
>>
>>                                           
>>
>>                                          will fail to
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> output
>>
>>                                              >
>>
>>                                              >     >>> results.
>>
>>                                              >
>>
>>                                              >     >>> 2. If I emit this data using system time I have the
>>
>>                                          problem
>>
>>                                           
>>
>>                                          that when
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> reprocessing data my window may not be ten minutes
>>
>>                                          but 10
>>
>>                                           
>>
>>                                          hours if my
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> processing is very fast so it dramatically changes
>>
>>                                          the
>>
>>                                           
>>
>>                                          output.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>> Maybe a hybrid solution works: I window by event
>>
>>                                          time but
>>
>>                                           
>>
>>                                          trigger results
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> by system time for windows that have updated? Not
>>
>>                                          really
>>
>>                                           
>>
>>                                          sure the details
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> of making that work. Does that work? Are there
>>
>>                                          concrete
>>
>>                                           
>>
>>                                          examples where
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> you
>>
>>                                              >
>>
>>                                              >     >>> actually want the current behavior?
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>> -Jay
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>
>>                                           
>>
>>                                          arunmathew88@gmail.com
>>                                          <ma...@gmail.com>> <ar...@gmail.com>
>>                                          <ma...@gmail.com> <ar...@gmail.com>
>>                                          <ma...@gmail.com>
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> wrote:
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>>> Hi All,
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> Thanks for the KIP. We were also in need of a
>>
>>                                          mechanism to
>>
>>                                           
>>
>>                                          trigger
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>> punctuate in the absence of events.
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> As I described in [
>>
>>                                              >
>>
>>                                              >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>
>>                                              >
>>
>>                                              >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>
>>                                              >
>>
>>                                              >     >>>> plugin.system.issuetabpanels:comment-
>>
>>                                          tabpanel#comment-
>>
>>                                           
>>
>>                                          15926036
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>> ],
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>>    - Our approached involved using the event time
>>
>>                                          by
>>
>>                                           
>>
>>                                          default.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>    - The method to check if there is any punctuate
>>
>>                                          ready
>>
>>                                           
>>
>>                                          in the
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>    PunctuationQueue is triggered via the any event
>>
>>                                           
>>
>>                                          received by the
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> stream
>>
>>                                              >
>>
>>                                              >     >>>>    tread, or at the polling intervals in the
>>
>>                                          absence of
>>
>>                                           
>>
>>                                          any events.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>    - When we create Punctuate objects (which
>>
>>                                          contains the
>>
>>                                           
>>
>>                                          next event
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> time
>>
>>                                              >
>>
>>                                              >     >>>>    for punctuation and interval), we also record
>>
>>                                          the
>>
>>                                           
>>
>>                                          creation time
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> (system
>>
>>                                              >
>>
>>                                              >     >>>>    time).
>>
>>                                              >
>>
>>                                              >     >>>>    - While checking for maturity of Punctuate
>>
>>                                          Schedule by
>>
>>                                              >
>>
>>                                              >     >> mayBePunctuate
>>
>>                                              >
>>
>>                                              >     >>>>    method, we also check if the system clock has
>>
>>                                          elapsed
>>
>>                                           
>>
>>                                          the punctuate
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>    interval since the schedule creation time.
>>
>>                                              >
>>
>>                                              >     >>>>    - In the absence of any event, or in the
>>
>>                                          absence of any
>>
>>                                           
>>
>>                                          event for
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> one
>>
>>                                              >
>>
>>                                              >     >>>>    topic in the partition group assigned to the
>>
>>                                          stream
>>
>>                                           
>>
>>                                          task, the system
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>> time
>>
>>                                              >
>>
>>                                              >     >>>>    will elapse the interval and we trigger a
>>
>>                                          punctuate
>>
>>                                           
>>
>>                                          using the
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> expected
>>
>>                                              >
>>
>>                                              >     >>>>    punctuation event time.
>>
>>                                              >
>>
>>                                              >     >>>>    - we then create the next punctuation schedule
>>
>>                                          as
>>
>>                                           
>>
>>                                          punctuation event
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> time
>>
>>                                              >
>>
>>                                              >     >>>>    + punctuation interval, [again recording the
>>
>>                                          system
>>
>>                                           
>>
>>                                          time of creation
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> of
>>
>>                                              >
>>
>>                                              >     >>>> the
>>
>>                                              >
>>
>>                                              >     >>>>    schedule].
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> We call this a Hybrid Punctuate. Of course, this
>>
>>                                          approach
>>
>>                                           
>>
>>                                          has pros and
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>> cons.
>>
>>                                              >
>>
>>                                              >     >>>> Pros
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>>    - Punctuates will happen in <punctuate
>>
>>                                          interval> time
>>
>>                                           
>>
>>                                          duration at
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> max
>>
>>                                              >
>>
>>                                              >     >>> in
>>
>>                                              >
>>
>>                                              >     >>>>    terms of system time.
>>
>>                                              >
>>
>>                                              >     >>>>    - The semantics as a whole continues to revolve
>>
>>                                          around
>>
>>                                           
>>
>>                                          event time.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>    - We can use the old data [old timestamps] to
>>
>>                                          rerun any
>>
>>                                           
>>
>>                                          experiments
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> or
>>
>>                                              >
>>
>>                                              >     >>>>    tests.
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> Cons
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>>    - In case the  <punctuate interval> is not a
>>
>>                                          time
>>
>>                                           
>>
>>                                          duration [say
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> logical
>>
>>                                              >
>>
>>                                              >     >>>>    time/event count], then the approach might not
>>
>>                                          be
>>
>>                                           
>>
>>                                          meaningful.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>    - In case there is a case where we have to wait
>>
>>                                          for an
>>
>>                                           
>>
>>                                          actual event
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> from
>>
>>                                              >
>>
>>                                              >     >>>>    a low event rate partition in the partition
>>
>>                                          group, this
>>
>>                                           
>>
>>                                          approach
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> will
>>
>>                                              >
>>
>>                                              >     >>>> jump
>>
>>                                              >
>>
>>                                              >     >>>>    the gun.
>>
>>                                              >
>>
>>                                              >     >>>>    - in case the event processing cannot catch up
>>
>>                                          with the
>>
>>                                           
>>
>>                                          event rate
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> and
>>
>>                                              >
>>
>>                                              >     >>>>    the expected timestamp events gets queued for
>>
>>                                          long
>>
>>                                           
>>
>>                                          time, this
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> approach
>>
>>                                              >
>>
>>                                              >     >>>>    might jump the gun.
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> I believe the above approach and discussion goes
>>
>>                                          close to
>>
>>                                           
>>
>>                                          the approach
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> A.
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> -----------
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> I like the idea of having an even count based
>>
>>                                          punctuate.
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> -----------
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> I agree with the discussion around approach C,
>>
>>                                          that we
>>
>>                                           
>>
>>                                          should provide
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> the
>>
>>                                              >
>>
>>                                              >     >>>> user with the option to choose system time or
>>
>>                                          event time
>>
>>                                           
>>
>>                                          based
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> punctuates.
>>
>>                                              >
>>
>>                                              >     >>>> But I believe that the user predominantly wants to
>>
>>                                          use
>>
>>                                           
>>
>>                                          event time while
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> not
>>
>>                                              >
>>
>>                                              >     >>>> missing out on regular punctuates due to event
>>
>>                                          delays or
>>
>>                                           
>>
>>                                          event
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> absences.
>>
>>                                              >
>>
>>                                              >     >>>> Hence a complex punctuate option as Matthias
>>
>>                                          mentioned
>>
>>                                           
>>
>>                                          (quoted below)
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> would
>>
>>                                              >
>>
>>                                              >     >>>> be most apt.
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> "- We might want to add "complex" schedules later
>>
>>                                          on
>>
>>                                           
>>
>>                                          (like, punctuate
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> on
>>
>>                                              >
>>
>>                                              >     >>>> every 10 seconds event-time or 60 seconds system-
>>
>>                                          time
>>
>>                                           
>>
>>                                          whatever comes
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>> first)."
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> -----------
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> I think I read somewhere that Kafka Streams
>>
>>                                          started with
>>
>>                                           
>>
>>                                          System Time as
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> the
>>
>>                                              >
>>
>>                                              >     >>>> punctuation standard, but was later changed to
>>
>>                                          Event Time.
>>
>>                                           
>>
>>                                          I guess
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> there
>>
>>                                              >
>>
>>                                              >     >>>> would be some good reason behind it. As Kafka
>>
>>                                          Streams want
>>
>>                                           
>>
>>                                          to evolve
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> more
>>
>>                                              >
>>
>>                                              >     >>>> on the Stream Processing front, I believe the
>>
>>                                          emphasis on
>>
>>                                           
>>
>>                                          event time
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> would
>>
>>                                              >
>>
>>                                              >     >>>> remain quite strong.
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> With Regards,
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> Arun Mathew
>>
>>                                              >
>>
>>                                              >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>
>>                                           
>>
>>                                          tobecker@tivo.com
>>                                          <ma...@tivo.com>> <to...@tivo.com>
>>                                          <ma...@tivo.com> <to...@tivo.com>
>>                                          <ma...@tivo.com>
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> wrote:
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>>> Yeah I like PuncutationType much better; I just
>>
>>                                          threw
>>
>>                                           
>>
>>                                          Time out there
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>> more as a strawman than an actual suggestion ;) I
>>
>>                                          still
>>
>>                                           
>>
>>                                          think it's
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>> worth considering what this buys us over an
>>
>>                                          additional
>>
>>                                           
>>
>>                                          callback. I
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>> foresee a number of punctuate implementations
>>
>>                                          following
>>
>>                                           
>>
>>                                          this pattern:
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>> public void punctuate(PunctuationType type) {
>>
>>                                              >
>>
>>                                              >     >>>>>     switch (type) {
>>
>>                                              >
>>
>>                                              >     >>>>>         case EVENT_TIME:
>>
>>                                              >
>>
>>                                              >     >>>>>             methodA();
>>
>>                                              >
>>
>>                                              >     >>>>>             break;
>>
>>                                              >
>>
>>                                              >     >>>>>         case SYSTEM_TIME:
>>
>>                                              >
>>
>>                                              >     >>>>>             methodB();
>>
>>                                              >
>>
>>                                              >     >>>>>             break;
>>
>>                                              >
>>
>>                                              >     >>>>>     }
>>
>>                                              >
>>
>>                                              >     >>>>> }
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>> I guess one advantage of this approach is we
>>
>>                                          could add
>>
>>                                           
>>
>>                                          additional
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>> punctuation types later in a backwards compatible
>>
>>                                          way
>>
>>                                           
>>
>>                                          (like event
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> count
>>
>>                                              >
>>
>>                                              >     >>>>> as you mentioned).
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>> -Tommy
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>
>>                                          Sax wrote:
>>
>>                                              >
>>
>>                                              >     >>>>>> That sounds promising.
>>
>>                                              >
>>
>>                                              >     >>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>> I am just wondering if `Time` is the best name.
>>
>>                                          Maybe we
>>
>>                                           
>>
>>                                          want to
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> add
>>
>>                                              >
>>
>>                                              >     >>>>>> other non-time based punctuations at some point
>>
>>                                          later. I
>>
>>                                           
>>
>>                                          would
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>> suggest
>>
>>                                              >
>>
>>                                              >     >>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>> enum PunctuationType {
>>
>>                                              >
>>
>>                                              >     >>>>>>   EVENT_TIME,
>>
>>                                              >
>>
>>                                              >     >>>>>>   SYSTEM_TIME,
>>
>>                                              >
>>
>>                                              >     >>>>>> }
>>
>>                                              >
>>
>>                                              >     >>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>> or similar. Just to keep the door open -- it's
>>
>>                                          easier to
>>
>>                                           
>>
>>                                          add new
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>> stuff
>>
>>                                              >
>>
>>                                              >     >>>>>> if the name is more generic.
>>
>>                                              >
>>
>>                                              >     >>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>> -Matthias
>>
>>                                              >
>>
>>                                              >     >>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>> I agree that the framework providing and
>>
>>                                          managing the
>>
>>                                           
>>
>>                                          notion of
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> stream
>>
>>                                              >
>>
>>                                              >     >>>>>>> time is valuable and not something we would
>>
>>                                          want to
>>
>>                                           
>>
>>                                          delegate to
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> the
>>
>>                                              >
>>
>>                                              >     >>>>>>> tasks. I'm not entirely convinced that a
>>
>>                                          separate
>>
>>                                           
>>
>>                                          callback
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> (option
>>
>>                                              >
>>
>>                                              >     >>>>>>> C)
>>
>>                                              >
>>
>>                                              >     >>>>>>> is that messy (it could just be a default
>>
>>                                          method with
>>
>>                                           
>>
>>                                          an empty
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> implementation), but if we wanted a single API
>>
>>                                          to
>>
>>                                           
>>
>>                                          handle both
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> cases,
>>
>>                                              >
>>
>>                                              >     >>>>>>> how about something like the following?
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>> enum Time {
>>
>>                                              >
>>
>>                                              >     >>>>>>>    STREAM,
>>
>>                                              >
>>
>>                                              >     >>>>>>>    CLOCK
>>
>>                                              >
>>
>>                                              >     >>>>>>> }
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>> Then on ProcessorContext:
>>
>>                                              >
>>
>>                                              >     >>>>>>> context.schedule(Time time, long interval)  //
>>
>>                                          We could
>>
>>                                           
>>
>>                                          allow
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> this
>>
>>                                              >
>>
>>                                              >     >>>>>>> to
>>
>>                                              >
>>
>>                                              >     >>>>>>> be called once for each value of time to mix
>>
>>                                           
>>
>>                                          approaches.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>> Then the Processor API becomes:
>>
>>                                              >
>>
>>                                              >     >>>>>>> punctuate(Time time) // time here denotes which
>>
>>                                           
>>
>>                                          schedule resulted
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> in
>>
>>                                              >
>>
>>                                              >     >>>>>>> this call.
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>> Thoughts?
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>
>>                                          Sax
>>
>>                                           
>>
>>                                          wrote:
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> Thanks a lot for the KIP Michal,
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> I was thinking about the four options you
>>
>>                                          proposed in
>>
>>                                           
>>
>>                                          more
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> details
>>
>>                                              >
>>
>>                                              >     >>>>>>>> and
>>
>>                                              >
>>
>>                                              >     >>>>>>>> this are my thoughts:
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> (A) You argue, that users can still
>>
>>                                          "punctuate" on
>>
>>                                           
>>
>>                                          event-time
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> via
>>
>>                                              >
>>
>>                                              >     >>>>>>>> process(), but I am not sure if this is
>>
>>                                          possible.
>>
>>                                           
>>
>>                                          Note, that
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> users
>>
>>                                              >
>>
>>                                              >     >>>>>>>> only
>>
>>                                              >
>>
>>                                              >     >>>>>>>> get record timestamps via context.timestamp().
>>
>>                                          Thus,
>>
>>                                           
>>
>>                                          users
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> would
>>
>>                                              >
>>
>>                                              >     >>>>>>>> need
>>
>>                                              >
>>
>>                                              >     >>>>>>>> to
>>
>>                                              >
>>
>>                                              >     >>>>>>>> track the time progress per partition (based
>>
>>                                          on the
>>
>>                                           
>>
>>                                          partitions
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> they
>>
>>                                              >
>>
>>                                              >     >>>>>>>> obverse via context.partition(). (This alone
>>
>>                                          puts a
>>
>>                                           
>>
>>                                          huge burden
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> on
>>
>>                                              >
>>
>>                                              >     >>>>>>>> the
>>
>>                                              >
>>
>>                                              >     >>>>>>>> user by itself.) However, users are not
>>
>>                                          notified at
>>
>>                                           
>>
>>                                          startup
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> what
>>
>>                                              >
>>
>>                                              >     >>>>>>>> partitions are assigned, and user are not
>>
>>                                          notified
>>
>>                                           
>>
>>                                          when
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> partitions
>>
>>                                              >
>>
>>                                              >     >>>>>>>> get
>>
>>                                              >
>>
>>                                              >     >>>>>>>> revoked. Because this information is not
>>
>>                                          available,
>>
>>                                           
>>
>>                                          it's not
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> possible
>>
>>                                              >
>>
>>                                              >     >>>>>>>> to
>>
>>                                              >
>>
>>                                              >     >>>>>>>> "manually advance" stream-time, and thus
>>
>>                                          event-time
>>
>>                                           
>>
>>                                          punctuation
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> within
>>
>>                                              >
>>
>>                                              >     >>>>>>>> process() seems not to be possible -- or do
>>
>>                                          you see a
>>
>>                                           
>>
>>                                          way to
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> get
>>
>>                                              >
>>
>>                                              >     >>>>>>>> it
>>
>>                                              >
>>
>>                                              >     >>>>>>>> done? And even if, it might still be too
>>
>>                                          clumsy to
>>
>>                                           
>>
>>                                          use.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> (B) This does not allow to mix both
>>
>>                                          approaches, thus
>>
>>                                           
>>
>>                                          limiting
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> what
>>
>>                                              >
>>
>>                                              >     >>>>>>>> users
>>
>>                                              >
>>
>>                                              >     >>>>>>>> can do.
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> (C) This should give all flexibility we need.
>>
>>                                          However,
>>
>>                                           
>>
>>                                          just
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> adding
>>
>>                                              >
>>
>>                                              >     >>>>>>>> one
>>
>>                                              >
>>
>>                                              >     >>>>>>>> more method seems to be a solution that is too
>>
>>                                          simple
>>
>>                                           
>>
>>                                          (cf my
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> comments
>>
>>                                              >
>>
>>                                              >     >>>>>>>> below).
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>
>>                                          sure how
>>
>>                                           
>>
>>                                          a user
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> could
>>
>>                                              >
>>
>>                                              >     >>>>>>>> enable system-time and event-time punctuation
>>
>>                                          in
>>
>>                                           
>>
>>                                          parallel.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> Overall options (C) seems to be the most
>>
>>                                          promising
>>
>>                                           
>>
>>                                          approach to
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> me.
>>
>>                                              >
>>
>>                                              >     >>>>>>>> Because I also favor a clean API, we might
>>
>>                                          keep
>>
>>                                           
>>
>>                                          current
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> punctuate()
>>
>>                                              >
>>
>>                                              >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>
>>                                          at some
>>
>>                                           
>>
>>                                          later
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> point
>>
>>                                              >
>>
>>                                              >     >>>>>>>> when
>>
>>                                              >
>>
>>                                              >     >>>>>>>> people use the "new punctuate API".
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> Couple of follow up questions:
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> - I am wondering, if we should have two
>>
>>                                          callback
>>
>>                                           
>>
>>                                          methods or
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> just
>>
>>                                              >
>>
>>                                              >     >>>>>>>> one
>>
>>                                              >
>>
>>                                              >     >>>>>>>> (ie, a unified for system and event time
>>
>>                                          punctuation
>>
>>                                           
>>
>>                                          or one for
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> each?).
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> - If we have one, how can the user figure out,
>>
>>                                          which
>>
>>                                           
>>
>>                                          condition
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> did
>>
>>                                              >
>>
>>                                              >     >>>>>>>> trigger?
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> - How would the API look like, for registering
>>
>>                                           
>>
>>                                          different
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> punctuate
>>
>>                                              >
>>
>>                                              >     >>>>>>>> schedules? The "type" must be somehow defined?
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> - We might want to add "complex" schedules
>>
>>                                          later on
>>
>>                                           
>>
>>                                          (like,
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> punctuate
>>
>>                                              >
>>
>>                                              >     >>>>>>>> on
>>
>>                                              >
>>
>>                                              >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>
>>                                          system-time
>>
>>                                           
>>
>>                                          whatever
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> comes
>>
>>                                              >
>>
>>                                              >     >>>>>>>> first). I don't say we should add this right
>>
>>                                          away, but
>>
>>                                           
>>
>>                                          we might
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> want
>>
>>                                              >
>>
>>                                              >     >>>>>>>> to
>>
>>                                              >
>>
>>                                              >     >>>>>>>> define the API in a way, that it allows
>>
>>                                          extensions
>>
>>                                           
>>
>>                                          like this
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> later
>>
>>                                              >
>>
>>                                              >     >>>>>>>> on,
>>
>>                                              >
>>
>>                                              >     >>>>>>>> without redesigning the API (ie, the API
>>
>>                                          should be
>>
>>                                           
>>
>>                                          designed
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> extensible)
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> - Did you ever consider count-based
>>
>>                                          punctuation?
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> I understand, that you would like to solve a
>>
>>                                          simple
>>
>>                                           
>>
>>                                          problem,
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> but
>>
>>                                              >
>>
>>                                              >     >>>>>>>> we
>>
>>                                              >
>>
>>                                              >     >>>>>>>> learned from the past, that just "adding some
>>
>>                                          API"
>>
>>                                           
>>
>>                                          quickly
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> leads
>>
>>                                              >
>>
>>                                              >     >>>>>>>> to a
>>
>>                                              >
>>
>>                                              >     >>>>>>>> not very well defined API that needs time
>>
>>                                          consuming
>>
>>                                           
>>
>>                                          clean up
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>> later on
>>
>>                                              >
>>
>>                                              >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>
>>                                          holistic
>>
>>                                              >
>>
>>                                              >     >>>>>>>> punctuation
>>
>>                                              >
>>
>>                                              >     >>>>>>>> KIP
>>
>>                                              >
>>
>>                                              >     >>>>>>>> with this from the beginning on to avoid later
>>
>>                                          painful
>>
>>                                              >
>>
>>                                              >     >> redesign.
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> -Matthias
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> Thanks Thomas,
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> I'm also wary of changing the existing
>>
>>                                          semantics of
>>
>>                                              >
>>
>>                                              >     >> punctuate,
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> for
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> backward compatibility reasons, although I
>>
>>                                          like the
>>
>>                                              >
>>
>>                                              >     >> conceptual
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> simplicity of that option.
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> Adding a new method to me feels safer but, in
>>
>>                                          a way,
>>
>>                                           
>>
>>                                          uglier.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> I
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> added
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> this to the KIP now as option (C).
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>
>>                                          more
>>
>>                                           
>>
>>                                          flexible,
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> as
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> it
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> allows
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> you to return any value, you're not limited
>>
>>                                          to event
>>
>>                                           
>>
>>                                          time or
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> system
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> time
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> (although I don't see an actual use case
>>
>>                                          where you
>>
>>                                           
>>
>>                                          might need
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> anything
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> else then those two). Hence I also proposed
>>
>>                                          the
>>
>>                                           
>>
>>                                          option to
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> allow
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> users
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> to, effectively, decide what "stream time" is
>>
>>                                          for
>>
>>                                           
>>
>>                                          them given
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> the
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> presence or absence of messages, much like
>>
>>                                          they can
>>
>>                                           
>>
>>                                          decide
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> what
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> msg
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> time
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> means for them using the TimestampExtractor.
>>
>>                                          What do
>>
>>                                           
>>
>>                                          you
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> think
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> about
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> that? This is probably most flexible but also
>>
>>                                          most
>>
>>                                              >
>>
>>                                              >     >> complicated.
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> All comments appreciated.
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> Cheers,
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> Michal
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> Although I fully agree we need a way to
>>
>>                                          trigger
>>
>>                                           
>>
>>                                          periodic
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> processing
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> that is independent from whether and when
>>
>>                                          messages
>>
>>                                           
>>
>>                                          arrive,
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> I'm
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> not sure
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> I like the idea of changing the existing
>>
>>                                          semantics
>>
>>                                           
>>
>>                                          across
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> the
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> board.
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> What if we added an additional callback to
>>
>>                                          Processor
>>
>>                                           
>>
>>                                          that
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> can
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> be
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>
>>                                          always
>>
>>                                           
>>
>>                                          called at
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> fixed, wall
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>
>>                                          have to
>>
>>                                           
>>
>>                                          give
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> up
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> the
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> notion
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> of stream time to be able to do periodic
>>
>>                                          processing.
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>
>>                                          Borowiecki
>>
>>                                           
>>
>>                                          wrote:
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> Hi all,
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>
>>                                           
>>
>>                                          punctuate
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> semantics
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> <https://cwiki.apache.org/
>>
>>                                           
>>
>>                                          confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
>>                                          confluence/display/KAFKA/KIP->
>>                                          <https://cwiki.apache.org/confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>                                          <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI
>>                                          P->
>>                                          <https://cwiki.apache.org/confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>                                          <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>
>>                                           
>>
>>                                          138%
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> 3A+C
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> hange+
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> punctuate+semantics>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> .
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> Appreciating there can be different views
>>
>>                                          on
>>
>>                                           
>>
>>                                          system-time
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> vs
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> event-
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> time
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> semantics for punctuation depending on use-
>>
>>                                          case and
>>
>>                                           
>>
>>                                          the
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> importance of
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> backwards compatibility of any such change,
>>
>>                                          I've
>>
>>                                           
>>
>>                                          left it
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> quite
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> open
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> and
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> hope to fill in more info as the discussion
>>
>>                                           
>>
>>                                          progresses.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> Thanks,
>>
>>                                              >
>>
>>                                              >     >>>>>>>>>>> Michal
>>
>>                                              >
>>
>>                                              >     >>>>>>> --
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>     Tommy Becker
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>     Senior Software Engineer
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>     tivo.com
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>> ________________________________
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>>> This email and any attachments may contain
>>
>>                                          confidential
>>
>>                                           
>>
>>                                          and
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> privileged material for the sole use of the
>>
>>                                          intended
>>
>>                                           
>>
>>                                          recipient.
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> Any
>>
>>                                              >
>>
>>                                              >     >>>>>>> review, copying, or distribution of this email
>>
>>                                          (or any
>>
>>                                              >
>>
>>                                              >     >> attachments)
>>
>>                                              >
>>
>>                                              >     >>>>>>> by others is prohibited. If you are not the
>>
>>                                          intended
>>
>>                                           
>>
>>                                          recipient,
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> please contact the sender immediately and
>>
>>                                          permanently
>>
>>                                           
>>
>>                                          delete this
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> email and any attachments. No employee or agent
>>
>>                                          of TiVo
>>
>>                                           
>>
>>                                          Inc. is
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> authorized to conclude any binding agreement on
>>
>>                                          behalf
>>
>>                                           
>>
>>                                          of TiVo
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> Inc.
>>
>>                                              >
>>
>>                                              >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>
>>                                          only be
>>
>>                                           
>>
>>                                          made by a
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>>>> signed written agreement.
>>
>>                                              >
>>
>>                                              >     >>>>>>>
>>
>>                                              >
>>
>>                                              >     >>>>> --
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>     Tommy Becker
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>     Senior Software Engineer
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>     tivo.com
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>> ________________________________
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>> This email and any attachments may contain
>>
>>                                          confidential
>>
>>                                           
>>
>>                                          and
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> privileged
>>
>>                                              >
>>
>>                                              >     >>>>> material for the sole use of the intended
>>
>>                                          recipient. Any
>>
>>                                           
>>
>>                                          review,
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>> copying,
>>
>>                                              >
>>
>>                                              >     >>>>> or distribution of this email (or any
>>
>>                                          attachments) by
>>
>>                                           
>>
>>                                          others is
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>> prohibited.
>>
>>                                              >
>>
>>                                              >     >>>>> If you are not the intended recipient, please
>>
>>                                          contact the
>>
>>                                           
>>
>>                                          sender
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>> immediately and permanently delete this email and
>>
>>                                          any
>>
>>                                           
>>
>>                                          attachments. No
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>> employee or agent of TiVo Inc. is authorized to
>>
>>                                          conclude
>>
>>                                           
>>
>>                                          any binding
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >>>>> agreement on behalf of TiVo Inc. by email.
>>
>>                                          Binding
>>
>>                                           
>>
>>                                          agreements with
>>
>>                                           
>>
>>                                              >
>>
>>                                              >     >> TiVo
>>
>>                                              >
>>
>>                                              >     >>>>> Inc. may only be made by a signed written
>>
>>                                          agreement.
>>
>>                                              >
>>
>>                                              >     >>>>>
>>
>>                                              >
>>
>>                                              >     >>>>
>>
>>                                              >
>>
>>                                              >     >>>
>>
>>                                              >
>>
>>                                              >     >>
>>
>>                                              >
>>
>>                                              >     >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > --
>>
>>                                              >
>>
>>                                              > <http://www.openbet.com/>
>>                                          <http://www.openbet.com/> <http://www.openbet.com/>
>>                                          <http://www.openbet.com/>
>>
>>                                           
>>
>>                                              >
>>
>>                                              > *Michal Borowiecki*
>>
>>                                              >
>>
>>                                              > *Senior Software Engineer L4*
>>
>>                                              >
>>
>>                                              > *T: *
>>
>>                                              >
>>
>>                                              > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>
>>                                              >
>>
>>                                              > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > *E: *
>>
>>                                              >
>>
>>                                              > michal.borowiecki@openbet.com
>>                                          <ma...@openbet.com>
>>
>>                                              >
>>
>>                                              > *W: *
>>
>>                                              >
>>
>>                                              > www.openbet.com
>>                                          <http://www.openbet.com>
>>
>>                                              >
>>
>>                                              > *OpenBet Ltd*
>>
>>                                              >
>>
>>                                              > Chiswick Park Building 9
>>
>>                                              >
>>
>>                                              > 566 Chiswick High Rd
>>
>>                                              >
>>
>>                                              > London
>>
>>                                              >
>>
>>                                              > W4 5XT
>>
>>                                              >
>>
>>                                              > UK
>>
>>                                              >
>>
>>                                              > <https://www.openbet.com/email_promo>
>>                                          <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>                                          <https://www.openbet.com/email_promo>
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                              > This message is confidential and intended only for the
>>
>>                                          addressee.
>>
>>                                           
>>
>>                                          If you
>>
>>                                           
>>
>>                                              > have received this message in error, please immediately
>>
>>                                          notify the
>>
>>                                              > postmaster@openbet.com
>>                                          <ma...@openbet.com> and delete it from your system as
>>
>>                                          well as
>>
>>                                           
>>
>>                                          any
>>
>>                                           
>>
>>                                              > copies. The content of e-mails as well as traffic data may
>>
>>                                          be
>>
>>                                           
>>
>>                                          monitored by
>>
>>                                           
>>
>>                                              > OpenBet for employment and security purposes. To protect
>>
>>                                          the
>>
>>                                           
>>
>>                                          environment
>>
>>                                           
>>
>>                                              > please do not print this e-mail unless necessary. OpenBet
>>
>>                                          Ltd.
>>
>>                                           
>>
>>                                          Registered
>>
>>                                           
>>
>>                                              > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>
>>                                          London,
>>
>>                                           
>>
>>                                          W4 5XT,
>>
>>                                           
>>
>>                                              > United Kingdom. A company registered in England and Wales.
>>
>>                                           
>>
>>                                          Registered no.
>>
>>                                           
>>
>>                                              > 3134634. VAT no. GB927523612
>>
>>                                              >
>>
>>                                              >
>>
>>                                              >
>>
>>                                           
>>
>>                                           
>>
>>                                           
>>
>>                                          --
>>
>>                                           
>>
>>                                           
>>
>>                                              Tommy Becker
>>
>>                                           
>>
>>                                              Senior Software Engineer
>>
>>                                           
>>
>>                                              O +1 919.460.4747 <%28919%29%20460-4747>
>>
>>                                           
>>
>>                                           
>>
>>                                              tivo.com
>>
>>                                           
>>
>>                                           
>>
>>                                           
>>
>>                                          ________________________________
>>
>>                                           
>>
>>                                          This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>
>>                                           
>>
>>                                           
>>
>>                                          --
>>
>>                                          <http://www.openbet.com/>
>>                                          <http://www.openbet.com/> Michal Borowiecki
>>
>>                                          Senior Software Engineer L4
>>
>>                                          T: +44 208 742 1600 <+44%2020%208742%201600>
>>
>>                                           
>>
>>                                           
>>
>>                                           
>>
>>                              --
>>
>>                              Signature
>>
>>                              <http://www.openbet.com/>
>>                              <http://www.openbet.com/>     Michal Borowiecki
>>
>>                              Senior Software Engineer L4
>>
>>                                  T:     +44 208 742 1600
>>
>>                               
>>
>>                                  
>>
>>                                  +44 203 249 8448
>>
>>                               
>>
>>                                  
>>
>>                                   
>>
>>                                 E:  michal.borowiecki@openbet.com
>>                              <ma...@openbet.com>
>>
>>                                 W:  www.openbet.com <http://www.openbet.com> <http://www.openbet.com/>
>>                              <http://www.openbet.com/>
>>
>>                               
>>
>>                                  
>>
>>                                  OpenBet Ltd
>>
>>                               
>>
>>                                  Chiswick Park Building 9
>>
>>                               
>>
>>                                  566 Chiswick High Rd
>>
>>                               
>>
>>                                  London
>>
>>                               
>>
>>                                  W4 5XT
>>
>>                               
>>
>>                                  UK
>>
>>                               
>>
>>                                  
>>
>>                              <https://www.openbet.com/email_promo>
>>                              <https://www.openbet.com/email_promo>
>>
>>                               
>>
>>                              This message is confidential and intended only for the addressee. If
>>
>>                              you have received this message in error, please immediately notify the
>>
>>                              postmaster@openbet.com
>>                              <ma...@openbet.com> <ma...@openbet.com>
>>                              <ma...@openbet.com> and delete it
>>
>>                              from your system as well as any copies. The content of e-mails as well
>>
>>                              as traffic data may be monitored by OpenBet for employment and
>>
>>                              security purposes. To protect the environment please do not print this
>>
>>                              e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>
>>                              Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>
>>                              company registered in England and Wales. Registered no. 3134634. VAT
>>
>>                              no. GB927523612
>>
>>                               
>>
>>                          --
>>
>>                          Signature
>>
>>                          <http://www.openbet.com/> <http://www.openbet.com/>   Michal Borowiecki
>>
>>                          Senior Software Engineer L4
>>
>>                            T:      +44 208 742 1600
>>
>>                           
>>
>>                            
>>
>>                            +44 203 249 8448
>>
>>                           
>>
>>                            
>>
>>                             
>>
>>                            E:  michal.borowiecki@openbet.com
>>                          <ma...@openbet.com>
>>
>>                            W:   www.openbet.com <http://www.openbet.com> <http://www.openbet.com/> <http://www.openbet.com/>
>>
>>                           
>>
>>                            
>>
>>                            OpenBet Ltd
>>
>>                           
>>
>>                            Chiswick Park Building 9
>>
>>                           
>>
>>                            566 Chiswick High Rd
>>
>>                           
>>
>>                            London
>>
>>                           
>>
>>                            W4 5XT
>>
>>                           
>>
>>                            UK
>>
>>                           
>>
>>                            
>>
>>                          <https://www.openbet.com/email_promo>
>>                          <https://www.openbet.com/email_promo>
>>
>>                           
>>
>>                          This message is confidential and intended only for the addressee. If you
>>
>>                          have received this message in error, please immediately notify the
>>
>>                          postmaster@openbet.com
>>                          <ma...@openbet.com> <ma...@openbet.com>
>>                          <ma...@openbet.com> and delete it
>>
>>                          from your system as well as any copies. The content of e-mails as well
>>
>>                          as traffic data may be monitored by OpenBet for employment and security
>>
>>                          purposes. To protect the environment please do not print this e-mail
>>
>>                          unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>
>>                          9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>
>>                          registered in England and Wales. Registered no. 3134634. VAT no.
>>
>>                          GB927523612
>>
>>                           
>>
>>                  --
>>
>>                  Signature
>>
>>                  <http://www.openbet.com/> <http://www.openbet.com/>    Michal Borowiecki
>>
>>                  Senior Software Engineer L4
>>
>>                         T:      +44 208 742 1600
>>
>>                   
>>
>>                         
>>
>>                         +44 203 249 8448
>>
>>                   
>>
>>                         
>>
>>                          
>>
>>                         E:        michal.borowiecki@openbet.com
>>                  <ma...@openbet.com>
>>
>>                         W:      www.openbet.com <http://www.openbet.com> <http://www.openbet.com/> <http://www.openbet.com/>
>>
>>                   
>>
>>                         
>>
>>                         OpenBet Ltd
>>
>>                   
>>
>>                         Chiswick Park Building 9
>>
>>                   
>>
>>                         566 Chiswick High Rd
>>
>>                   
>>
>>                         London
>>
>>                   
>>
>>                         W4 5XT
>>
>>                   
>>
>>                         UK
>>
>>                   
>>
>>                         
>>
>>                  <https://www.openbet.com/email_promo>
>>                  <https://www.openbet.com/email_promo>
>>
>>                   
>>
>>                  This message is confidential and intended only for the addressee. If you
>>
>>                  have received this message in error, please immediately notify the
>>
>>                  postmaster@openbet.com <ma...@openbet.com> <ma...@openbet.com>
>>                  <ma...@openbet.com> and delete it
>>
>>                  from your system as well as any copies. The content of e-mails as well
>>
>>                  as traffic data may be monitored by OpenBet for employment and security
>>
>>                  purposes. To protect the environment please do not print this e-mail
>>
>>                  unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>
>>                  9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>
>>                  registered in England and Wales. Registered no. 3134634. VAT no.
>>
>>                  GB927523612
>>
>>                   
>>
>>          --
>>
>>          Signature
>>
>>          <http://www.openbet.com/> <http://www.openbet.com/>  Michal Borowiecki
>>
>>          Senior Software Engineer L4
>>
>>             T:      +44 208 742 1600
>>
>>           
>>
>>             
>>
>>             +44 203 249 8448
>>
>>           
>>
>>             
>>
>>              
>>
>>             E:      michal.borowiecki@openbet.com <ma...@openbet.com>
>>
>>             W:      www.openbet.com <http://www.openbet.com> <http://www.openbet.com/> <http://www.openbet.com/>
>>
>>           
>>
>>             
>>
>>             OpenBet Ltd
>>
>>           
>>
>>             Chiswick Park Building 9
>>
>>           
>>
>>             566 Chiswick High Rd
>>
>>           
>>
>>             London
>>
>>           
>>
>>             W4 5XT
>>
>>           
>>
>>             UK
>>
>>           
>>
>>             
>>
>>          <https://www.openbet.com/email_promo>
>>          <https://www.openbet.com/email_promo>
>>
>>           
>>
>>          This message is confidential and intended only for the addressee. If you
>>
>>          have received this message in error, please immediately notify the
>>
>>          postmaster@openbet.com <ma...@openbet.com> <ma...@openbet.com> <ma...@openbet.com> and delete it
>>
>>          from your system as well as any copies. The content of e-mails as well
>>
>>          as traffic data may be monitored by OpenBet for employment and security
>>
>>          purposes. To protect the environment please do not print this e-mail
>>
>>          unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>
>>          9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>
>>          registered in England and Wales. Registered no. 3134634. VAT no.
>>
>>          GB927523612
>>
>>           
>>
>>   
>>
>> -- 
>>
>> <http://www.openbet.com/>
>>
>> 	
>>
>> *Michal Borowiecki*
>>
>> *Senior Software Engineer L4*
>>
>> 	
>>
>> *T: *
>>
>> 	
>>
>> +44 208 742 1600
>>
>> 		
>>
>> +44 203 249 8448
>>
>> 		
>>
>>   
>>
>> 	
>>
>> *E: *
>>
>> 	
>>
>> michal.borowiecki@openbet.com <ma...@openbet.com>
>>
>> 	
>>
>> *W: *
>>
>> 	
>>
>> www.openbet.com <http://www.openbet.com/>
>>
>> 	
>>
>> 	
>>
>> *OpenBet Ltd*
>>
>> 	
>>
>> Chiswick Park Building 9
>>
>> 	
>>
>> 566 Chiswick High Rd
>>
>> 	
>>
>> London
>>
>> 	
>>
>> W4 5XT
>>
>> 	
>>
>> UK
>>
>> 	
>>
>> <https://www.openbet.com/email_promo>
>>
>>   
>>
>> This message is confidential and intended only for the addressee. If you
>> have received this message in error, please immediately notify the
>> postmaster@openbet.com <ma...@openbet.com> and delete it
>> from your system as well as any copies. The content of e-mails as well
>> as traffic data may be monitored by OpenBet for employment and security
>> purposes. To protect the environment please do not print this e-mail
>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>> registered in England and Wales. Registered no. 3134634. VAT no. GB927523612
>>
>>   
>>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks for sharing Arun!

-Matthias

On 4/26/17 5:59 AM, Arun Mathew wrote:
> Hi All,
> 
>  
> 
> Apologies for the delayed response.
> 
> As requested by Tianji, I am sharing our implementation of hybrid punctuate.
> 
>  
> 
> It is a rather small patch, a hack for our use-case I would say. I
> suggest you all to have a look.
> 
> You can view/(download) the diff from
> https://github.com/apache/kafka/compare/0.10.1.1...arunmathew88:hybrid-punctuate
> (.diff)
> 
>  
> 
> Now I will try to address some of matters below
> 
>  
> 
>> In the current stream-time punctuation scheme, the first
> PunctuationSchedule is set to 0 + interval (what else, since we haven't
> seen any records yet), but when triggered, it passes the current stream
> time as timestamp argument and uses that to schedule the next
> punctuation. It all makes sense.
>> Arun, have you implemented any special handling for this case in your
> system?
> 
> In our implementation, we never send the system time as argument to
> punctuate, as it will be semantically incorrect to mix it. We use system
> time only to check if punctuate interval has elapsed in the wallclock.
> So in the situation of the 0 + interval as fist punctuation schedule,
> our hybrid punctuate implementation will call punctuate with
> (0+interval) as the argument, and (0 + interval  + interval) as the next
> punctuation schedule. Hence this flaw is not there.
> 
>  
> 
> However as I had mentioned in my original description mail (under the
> Cons section), and as pointed out by Matthias
> 
>  
> 
>>(2) IMHO the major disadvantage (issue?) of the hybrid approach is the
> implicit assumption that even-time progresses at the same "speed" as
> system-time during regular processing. This implies the assumption that
> a slower progress in stream-time indicates the absence of input events
> 
>  
> 
> Though this can be tweaked to detect a slow stream time progression to
> block hybrid punctuation, it won’t look very clean.
> 
>  
> 
> @Michal, regarding 2b
> 
>>but I can't believe this to be the only reason behind this choice as
> minimum is not the only function to guarantee the group partition time
> never going back. Using the largest or the average among partitions'
> timestamp would also guaranteed the group timestamp not going back as
> timestamp never goes back for any individual partition.
> 
> So why was minimum chosen? Is it depended on by window semantics
> somewhere or anything else?
> 
>  
> 
> My thought as to why minimum was chosen can be illustrated via a simple
> use-case. Say we are consuming data from topic A event{ts, valueA} and
> topic B  event{ts, valueB}, and output to topic C event{ts, valueC}. If
> we have to output the valueC = valueA + valueB for matching ts values,
> then we need to punctuate to happen after all values for a specified ts
> has arrived. (of course we are skipping the case of missing events,
> late/out of order events etc).
> 
>  
> 
>  
> 
>> I agree that expressing the intended hybrid semantics is harder if we
> 
> offer only #StreamTime and #SystemTime punctuation. However, I also
> 
> believe that the hybrid approach is a "false friend" with regard to
> 
> reasoning about the semantics (it indicates that it more easy as it is
> 
> in reality). Therefore, we might be better off to not offer the hybrid
> 
> approach and make it clear to a developed, that it is hard to mix
> 
> #StreamTime and #SystemTime in a semantically sound way.
> 
>  
> 
> I agree with Matthias, that hybrid approach might not be a clean
> approach, it has its own assumptions, In particular I don’t like that
> the approach will have to virtually force push the stream time, for a
> low event rate partition in the partition group, to trigger a punctuate.
>  Nevertheless, I believe it is quite handy in a few use-cases.
> 
>  
> 
> I totally agree with the manthra of keeping things simple and straight
> forward at the platform level and let the developer build things on top
> of it. So let us think how to modify the punctuate semantics so that the
> user can implement desired effect, stream time /system time /hybrid
> punctuate at the developer level. I had to patch the Kafka Streams
> because I wasn’t able to implement it using the levers and switches
> provided by the KS library. I had originally just wanted a pure system
> time based punctuate, like the cron case discussed before.
> 
>  
> 
>  
> 
>  
> 
> It was a long read, since my last mail, I will go through all
> conversations again and update the Use-Cases document appropriately.
> 
>  
> 
> --
> 
> With Regards
> 
>  
> 
> Arun Mathew
> 
> Yahoo! JAPAN Corporation
> 
>  
> 
>  
> 
> *From: *Michal Borowiecki <mi...@openbet.com>
> *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> *Date: *Tuesday, April 25, 2017 at 07:11
> *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
> 
>  
> 
> Sorry, my original point was lost or I didn't make it clear enough.
> 
> I don't challenge the need for a system-time interval upper bound, one
> we don't have now. On the contrary my own use-case depends on it.
> However, the hybrid does provide a system-time interval upper bound,
> even in the absence of records, it only fails to provide a lower bound.
> 
> As to heartbeats, IMO that wouldn't need a lower bound on the heartbeat
> interval, an upper bound should suffice.
> 
>  
> 
> Nevertheless, I think I found a flaw in the hybrid proposal, as follows:
> 
> In the current stream-time punctuation scheme, the first
> PunctuationSchedule is set to 0 + interval (what else, since we haven't
> seen any records yet), but when triggered, it passes the current stream
> time as timestamp argument and uses that to schedule the next
> punctuation. It all makes sense.
> 
> In the hybrid this would only work if the first punctuation is triggered
> by stream time advance but could have unintended consequences if the
> first triggering is by system time.
> 
> Imagine this scenario: my apps are down and I have loads of records to
> process (perhaps I reset offsets to reprocess data).
> 
> If I start my streams app but records don't start coming in immediately
> (perhaps there is another app upstream that I have yet to start), then
> the first punctuation will be triggered by system time. It won't have
> any stream-time to use so, the only timestamp it can use is the
> system-time timestamp, both as an argument to punctuate and for the next
> schedule.
> 
> But if then messages start flowing and have event time way in the past
> (assuming event/ingestion time extractor), then stream time will be
> behind system-time until the stream catches up (which may be never if I
> just want to run in historical data as an experiment/validation).
> However, punctuate (having been triggered first by system time) will
> keep on recurring at regular intervals and always passing in the system
> time (or to be precise: the system time of previous punctuation +
> interval). That is surely not what is desired or expected, given the
> steady inflow of records.
> 
> Arun, have you implemented any special handling for this case in your
> system?
> 
> As mentioned before the hybrid punctuation scheme is not easy to reason
> about, I can't deny that.
> 
>  
> 
> Happy to stick with the 2 PunctuationTypes proposal (stream time and
> system time). I'll try to write up how users can implement hybrid-like
> semantics themselves on top of that to address those demanding use-cases
> mentioned on the wiki.
> 
> Maybe it won't end up too complex after all, but as per the mantra, it's
> enough that it's made possible ;-)
> 
> Thanks,
> 
> Michal
> 
>  
> 
> On 24/04/17 18:22, Matthias J. Sax wrote:
> 
>             Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?
> 
>     If you go with stream-time and don't have any input records for all
> 
>     partitions, punctuate would not be called at all, and thus your
> 
>     dashboard would "freeze".
> 
>      
> 
>             I thought about cron-type things, but aren't they better triggered by an
> 
>             external scheduler (they're more flexible anyway), which then feeds
> 
>             "commands" into the topology?
> 
>     I guess it depends what kind of periodic action you want to trigger. The
> 
>     "cron job" was just an analogy. Maybe it's just a heartbeat to some
> 
>     other service, that signals that your Streams app is still running.
> 
>      
> 
>      
> 
>     -Matthias
> 
>      
> 
>      
> 
>     On 4/24/17 10:02 AM, Michal Borowiecki wrote:
> 
>         Thanks!
> 
>          
> 
>         Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?
> 
>          
> 
>         Unless too frequent messages on replay could overpower it?
> 
>          
> 
>          
> 
>         I thought about cron-type things, but aren't they better triggered by an
> 
>         external scheduler (they're more flexible anyway), which then feeds
> 
>         "commands" into the topology?
> 
>          
> 
>         Just my 2c.
> 
>          
> 
>         Cheers,
> 
>          
> 
>         Michal
> 
>          
> 
>          
> 
>         On 24/04/17 17:57, Matthias J. Sax wrote:
> 
>             A simple example would be some dashboard app, that needs to get
> 
>             "current" status in regular time intervals (ie, and real-time app).
> 
>              
> 
>             Or something like a "scheduler" -- think "cron job" application.
> 
>              
> 
>              
> 
>             -Matthias
> 
>              
> 
>             On 4/24/17 2:23 AM, Michal Borowiecki wrote:
> 
>                 Hi Matthias,
> 
>                  
> 
>                 I agree it's difficult to reason about the hybrid approach, I certainly
> 
>                 found it hard and I'm totally on board with the mantra.
> 
>                  
> 
>                 I'd be happy to limit the scope of this KIP to add system-time
> 
>                 punctuation semantics (in addition to existing stream-time semantics)
> 
>                 and leave more complex schemes for users to implement on top of that.
> 
>                  
> 
>                 Further additional PunctuationTypes, could then be added by future KIPs,
> 
>                 possibly including the hybrid approach once it has been given more thought.
> 
>                  
> 
>                     There are real-time applications, that want to get
> 
>                     callbacks in regular system-time intervals (completely independent from
> 
>                     stream-time).
> 
>                 Can you please describe what they are, so that I can put them on the
> 
>                 wiki for later reference?
> 
>                  
> 
>                 Thanks,
> 
>                  
> 
>                 Michal
> 
>                  
> 
>                  
> 
>                 On 23/04/17 21:27, Matthias J. Sax wrote:
> 
>                     Hi,
> 
>                      
> 
>                     I do like Damian's API proposal about the punctuation callback function.
> 
>                      
> 
>                     I also did reread the KIP and thought about the semantics we want to
> 
>                     provide.
> 
>                      
> 
>                         Given the above, I don't see a reason any more for a separate system-time based punctuation.
> 
>                     I disagree here. There are real-time applications, that want to get
> 
>                     callbacks in regular system-time intervals (completely independent from
> 
>                     stream-time). Thus we should allow this -- if we really follow the
> 
>                     "hybrid" approach, this could be configured with stream-time interval
> 
>                     infinite and delay whatever system-time punctuation interval you want to
> 
>                     have. However, I would like to add a proper API for this and do this
> 
>                     configuration under the hood (that would allow one implementation within
> 
>                     all kind of branching for different cases).
> 
>                      
> 
>                     Thus, we definitely should have PunctutionType#StreamTime and
> 
>                     #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
> 
>                     a fan of your latest API proposal.
> 
>                      
> 
>                      
> 
>                     About the hybrid approach in general. On the one hand I like it, on the
> 
>                     other hand, it seems to be rather (1) complicated (not necessarily from
> 
>                     an implementation point of view, but for people to understand it) and
> 
>                     (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
> 
>                      
> 
>                         It may appear complicated at first but I do think these semantics will
> 
>                         still be more understandable to users than having 2 separate punctuation
> 
>                         schedules/callbacks with different PunctuationTypes.
> 
>                     This statement only holds if you apply strong assumptions that I don't
> 
>                     believe hold in general -- see (2) for details -- and I think it is
> 
>                     harder than you assume to reason about the hybrid approach in general.
> 
>                     IMHO, the hybrid approach is a "false friend" that seems to be easy to
> 
>                     reason about...
> 
>                      
> 
>                      
> 
>                     (1) Streams always embraced "easy to use" and we should really be
> 
>                     careful to keep it this way. On the other hand, as we are talking about
> 
>                     changes to PAPI, it won't affect DSL users (DSL does not use punctuation
> 
>                     at all at the moment), and thus, the "easy to use" mantra might not be
> 
>                     affected, while it will allow advanced users to express more complex stuff.
> 
>                      
> 
>                     I like the mantra: "make simple thing easy and complex things possible".
> 
>                      
> 
>                     (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
> 
>                     implicit assumption that even-time progresses at the same "speed" as
> 
>                     system-time during regular processing. This implies the assumption that
> 
>                     a slower progress in stream-time indicates the absence of input events
> 
>                     (and that later arriving input events will have a larger event-time with
> 
>                     high probability). Even if this might be true for some use cases, I
> 
>                     doubt it holds in general. Assume that you get a spike in traffic and
> 
>                     for some reason stream-time does advance slowly because you have more
> 
>                     records to process. This might trigger a system-time based punctuation
> 
>                     call even if this seems not to be intended. I strongly believe that it
> 
>                     is not easy to reason about the semantics of the hybrid approach (even
> 
>                     if the intentional semantics would be super useful -- but I doubt that
> 
>                     we get want we ask for).
> 
>                      
> 
>                     Thus, I also believe that one might need different "configuration"
> 
>                     values for the hybrid approach if you run the same code for different
> 
>                     scenarios: regular processing, re-processing, catching up scenario. And
> 
>                     as the term "configuration" implies, we might be better off to not mix
> 
>                     configuration with business logic that is expressed via code.
> 
>                      
> 
>                      
> 
>                     One more comment: I also don't think that the hybrid approach is
> 
>                     deterministic as claimed in the use-case subpage. I understand the
> 
>                     reasoning and agree, that it is deterministic if certain assumptions
> 
>                     hold -- compare above -- and if configured correctly. But strictly
> 
>                     speaking it's not because there is a dependency on system-time (and
> 
>                     IMHO, if system-time is involved it cannot be deterministic by definition).
> 
>                      
> 
>                      
> 
>                         I see how in theory this could be implemented on top of the 2 punctuate
> 
>                         callbacks with the 2 different PunctuationTypes (one stream-time based,
> 
>                         the other system-time based) but it would be a much more complicated
> 
>                         scheme and I don't want to suggest that.
> 
>                     I agree that expressing the intended hybrid semantics is harder if we
> 
>                     offer only #StreamTime and #SystemTime punctuation. However, I also
> 
>                     believe that the hybrid approach is a "false friend" with regard to
> 
>                     reasoning about the semantics (it indicates that it more easy as it is
> 
>                     in reality). Therefore, we might be better off to not offer the hybrid
> 
>                     approach and make it clear to a developed, that it is hard to mix
> 
>                     #StreamTime and #SystemTime in a semantically sound way.
> 
>                      
> 
>                      
> 
>                     Looking forward to your feedback. :)
> 
>                      
> 
>                     -Matthias
> 
>                      
> 
>                      
> 
>                      
> 
>                      
> 
>                     On 4/22/17 11:43 AM, Michal Borowiecki wrote:
> 
>                         Hi all,
> 
>                          
> 
>                         Looking for feedback on the functional interface approach Damian
> 
>                         proposed. What do people think?
> 
>                          
> 
>                         Further on the semantics of triggering punctuate though:
> 
>                          
> 
>                         I ran through the 2 use cases that Arun had kindly put on the wiki
> 
>                         (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
> 
>                         in my head and on a whiteboard and I can't find a better solution than
> 
>                         the "hybrid" approach he had proposed.
> 
>                          
> 
>                         I see how in theory this could be implemented on top of the 2 punctuate
> 
>                         callbacks with the 2 different PunctuationTypes (one stream-time based,
> 
>                         the other system-time based) but it would be a much more complicated
> 
>                         scheme and I don't want to suggest that.
> 
>                          
> 
>                         However, to add to the hybrid algorithm proposed, I suggest one
> 
>                         parameter to that: a tolerance period, expressed in milliseconds
> 
>                         system-time, after which the punctuation will be invoked in case the
> 
>                         stream-time advance hasn't triggered it within the requested interval
> 
>                         since the last invocation of punctuate (as recorded in system-time)
> 
>                          
> 
>                         This would allow a user-defined tolerance for late arriving events. The
> 
>                         trade off would be left for the user to decide: regular punctuation in
> 
>                         the case of absence of events vs allowing for records arriving late or
> 
>                         some build-up due to processing not catching up with the event rate.
> 
>                         In the one extreme, this tolerance could be set to infinity, turning
> 
>                         hybrid into simply stream-time based punctuate, like we have now. In the
> 
>                         other extreme, the tolerance could be set to 0, resulting in a
> 
>                         system-time upper bound on the effective punctuation interval.
> 
>                          
> 
>                         Given the above, I don't see a reason any more for a separate
> 
>                         system-time based punctuation. The "hybrid" approach with 0ms tolerance
> 
>                         would under normal operation trigger at regular intervals wrt the
> 
>                         system-time, except in cases of re-play/catch-up, where the stream time
> 
>                         advances faster than system time. In these cases punctuate would happen
> 
>                         more often than the specified interval wrt system time. However, the
> 
>                         use-cases that need system-time punctuations (that I've seen at least)
> 
>                         really only have a need for an upper bound on punctuation delay but
> 
>                         don't need a lower bound.
> 
>                          
> 
>                         To that effect I'd propose the api to be as follows, on ProcessorContext:
> 
>                          
> 
>                         schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
> 
>                          
> 
>                         schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
> 
>                          
> 
>                         Punctuation is triggered when either:
> 
>                         - the stream time advances past the (stream time of the previous
> 
>                         punctuation) + interval;
> 
>                         - or (iff the toleranceInterval is set) when the system time advances
> 
>                         past the (system time of the previous punctuation) + interval +
> 
>                         toleranceInterval
> 
>                          
> 
>                         In either case:
> 
>                         - we trigger punctuate passing as the argument the stream time at which
> 
>                         the current punctuation was meant to happen
> 
>                         - next punctuate is scheduled at (stream time at which the current
> 
>                         punctuation was meant to happen) + interval
> 
>                          
> 
>                         It may appear complicated at first but I do think these semantics will
> 
>                         still be more understandable to users than having 2 separate punctuation
> 
>                         schedules/callbacks with different PunctuationTypes.
> 
>                          
> 
>                          
> 
>                          
> 
>                         PS. Having re-read this, maybe the following alternative would be easier
> 
>                         to understand (WDYT?):
> 
>                          
> 
>                         schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
> 
>                          
> 
>                         schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
> 
>                          
> 
>                         Punctuation is triggered when either:
> 
>                         - the stream time advances past the (stream time of the previous
> 
>                         punctuation) + streamTimeInterval;
> 
>                         - or (iff systemTimeUpperBound is set) when the system time advances
> 
>                         past the (system time of the previous punctuation) + systemTimeUpperBound
> 
>                          
> 
>                         Awaiting comments.
> 
>                          
> 
>                         Thanks,
> 
>                         Michal
> 
>                          
> 
>                         On 21/04/17 16:56, Michal Borowiecki wrote:
> 
>                             Yes, that's what I meant. Just wanted to highlight we'd deprecate it
> 
>                             in favour of something that doesn't return a record. Not a problem though.
> 
>                              
> 
>                              
> 
>                             On 21/04/17 16:32, Damian Guy wrote:
> 
>                                 Thanks Michal,
> 
>                                 I agree Transformer.punctuate should also be void, but we can deprecate
> 
>                                 that too in favor of the new interface.
> 
>                                  
> 
>                                 Thanks for the javadoc PR!
> 
>                                  
> 
>                                 Cheers,
> 
>                                 Damian
> 
>                                  
> 
>                                 On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
> 
>                                 michal.borowiecki@openbet.com
>                                 <ma...@openbet.com>> wrote:
> 
>                                  
> 
>                                     Yes, that looks better to me.
> 
>                                      
> 
>                                     Note that punctuate on Transformer is currently returning a record, but I
> 
>                                     think it's ok to have all output records be sent via
> 
>                                     ProcessorContext.forward, which has to be used anyway if you want to send
> 
>                                     multiple records from one invocation of punctuate.
> 
>                                      
> 
>                                     This way it's consistent between Processor and Transformer.
> 
>                                      
> 
>                                      
> 
>                                     BTW, looking at this I found a glitch in the javadoc and put a comment
> 
>                                     there:
> 
>                                      
> 
>                                     https://github.com/apache/kafka/pull/2413/files#r112634612
> 
>                                      
> 
>                                     and PR: https://github.com/apache/kafka/pull/2884
> 
>                                      
> 
>                                     Cheers,
> 
>                                      
> 
>                                     Michal
> 
>                                     On 20/04/17 18:55, Damian Guy wrote:
> 
>                                      
> 
>                                     Hi Michal,
> 
>                                      
> 
>                                     Thanks for the KIP. I'd like to propose a bit more of a radical change to
> 
>                                     the API.
> 
>                                     1. deprecate the punctuate method on Processor
> 
>                                     2. create a new Functional Interface just for Punctuation, something like:
> 
>                                     interface Punctuator {
> 
>                                         void punctuate(long timestamp)
> 
>                                     }
> 
>                                     3. add a new schedule function to ProcessorContext: schedule(long
> 
>                                     interval, PunctuationType type, Punctuator callback)
> 
>                                     4. deprecate the existing schedule function
> 
>                                      
> 
>                                     Thoughts?
> 
>                                      
> 
>                                     Thanks,
> 
>                                     Damian
> 
>                                      
> 
>                                     On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
> 
>                                     michal.borowiecki@openbet.com
>                                     <ma...@openbet.com>> wrote:
> 
>                                      
> 
>                                         Hi Thomas,
> 
>                                          
> 
>                                         I would say our use cases fall in the same category as yours.
> 
>                                          
> 
>                                         1) One is expiry of old records, it's virtually identical to yours.
> 
>                                          
> 
>                                         2) Second one is somewhat more convoluted but boils down to the same type
> 
>                                         of design:
> 
>                                          
> 
>                                         Incoming messages carry a number of fields, including a timestamp.
> 
>                                          
> 
>                                         Outgoing messages contain derived fields, one of them (X) is depended on
> 
>                                         by the timestamp input field (Y) and some other input field (Z).
> 
>                                          
> 
>                                         Since the output field X is derived in some non-trivial way, we don't
> 
>                                         want to force the logic onto downstream apps. Instead we want to calculate
> 
>                                         it in the Kafka Streams app, which means we re-calculate X as soon as the
> 
>                                         timestamp in Y is reached (wall clock time) and send a message if it
> 
>                                         changed (I say "if" because the derived field (X) is also conditional on
> 
>                                         another input field Z).
> 
>                                          
> 
>                                         So we have kv stores with the records and an additional kv store with
> 
>                                         timestamp->id mapping which act like an index where we periodically do a
> 
>                                         ranged query.
> 
>                                          
> 
>                                         Initially we naively tried doing it in punctuate which of course didn't
> 
>                                         work when there were no regular msgs on the input topic.
> 
>                                         Since this was before 0.10.1 and state stores weren't query-able from
> 
>                                         outside we created a "ticker" that produced msgs once per second onto
> 
>                                         another topic and fed it into the same topology to trigger punctuate.
> 
>                                         This didn't work either, which was much more surprising to us at the
> 
>                                         time, because it was not obvious at all that punctuate is only triggered if
> 
>                                         *all* input partitions receive messages regularly.
> 
>                                         In the end we had to break this into 2 separate Kafka Streams. Main
> 
>                                         transformer doesn't use punctuate but sends values of timestamp field Y and
> 
>                                         the id to a "scheduler" topic where also the periodic ticks are sent. This
> 
>                                         is consumed by the second topology and is its only input topic. There's a
> 
>                                         transformer on that topic which populates and updates the time-based
> 
>                                         indexes and polls them from punctuate. If the time in the timestamp
> 
>                                         elapsed, the record id is sent to the main transformer, which
> 
>                                         updates/deletes the record from the main kv store and forwards the
> 
>                                         transformed record to the output topic.
> 
>                                          
> 
>                                         To me this setup feels horrendously complicated for what it does.
> 
>                                          
> 
>                                         We could incrementally improve on this since 0.10.1 to poll the
> 
>                                         timestamp->id "index" stores from some code outside the KafkaStreams
> 
>                                         topology so that at least we wouldn't need the extra topic for "ticks".
> 
>                                         However, the ticks don't feel so hacky when you realise they give you
> 
>                                         some hypothetical benefits in predictability. You can reprocess the
> 
>                                         messages in a reproducible manner, since the topologies use event-time,
> 
>                                         just that the event time is simply the wall-clock time fed into a topic by
> 
>                                         the ticks. (NB in our use case we haven't yet found a need for this kind of
> 
>                                         reprocessing).
> 
>                                         To make that work though, we would have to have the stream time advance
> 
>                                         based on the presence of msgs on the "tick" topic, regardless of the
> 
>                                         presence of messages on the other input topic.
> 
>                                          
> 
>                                         Same as in the expiry use case, both the wall-clock triggered punctuate
> 
>                                         and the hybrid would work to simplify this a lot.
> 
>                                          
> 
>                                         3) Finally, I have a 3rd use case in the making but I'm still looking if
> 
>                                         we can achieve it using session windows instead. I'll keep you posted if we
> 
>                                         have to go with punctuate there too.
> 
>                                          
> 
>                                         Thanks,
> 
>                                         Michal
> 
>                                          
> 
>                                          
> 
>                                         On 11/04/17 20:52, Thomas Becker wrote:
> 
>                                          
> 
>                                         Here's an example that we currently have.  We have a streams processor
> 
>                                         that does a transform from one topic into another. One of the fields in
> 
>                                         the source topic record is an expiration time, and one of the functions
> 
>                                         of the processor is to ensure that expired records get deleted promptly
> 
>                                         after that time passes (typically days or weeks after the message was
> 
>                                         originally produced). To do that, the processor keeps a state store of
> 
>                                         keys and expiration times, iterates that store in punctuate(), and
> 
>                                         emits delete (null) records for expired items. This needs to happen at
> 
>                                         some minimum interval regardless of the incoming message rate of the
> 
>                                         source topic.
> 
>                                          
> 
>                                         In this scenario, the expiration of records is the primary function of
> 
>                                         punctuate, and therefore the key requirement is that the wall-clock
> 
>                                         measured time between punctuate calls have some upper-bound. So a pure
> 
>                                         wall-clock based schedule would be fine for our needs. But the proposed
> 
>                                         "hybrid" system would also be acceptable if that satisfies a broader
> 
>                                         range of use-cases.
> 
>                                          
> 
>                                         On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
> 
>                                          
> 
>                                         I apologize for the longer email below.  To my defense, it started
> 
>                                         out much
> 
>                                         shorter. :-)  Also, to be super-clear, I am intentionally playing
> 
>                                         devil's
> 
>                                         advocate for a number of arguments brought forth in order to help
> 
>                                         improve
> 
>                                         this KIP -- I am not implying I necessarily disagree with the
> 
>                                         arguments.
> 
>                                          
> 
>                                         That aside, here are some further thoughts.
> 
>                                          
> 
>                                         First, there are (at least?) two categories for actions/behavior you
> 
>                                         invoke
> 
>                                         via punctuate():
> 
>                                          
> 
>                                         1. For internal housekeeping of your Processor or Transformer (e.g.,
> 
>                                         to
> 
>                                         periodically commit to a custom store, to do metrics/logging).  Here,
> 
>                                         the
> 
>                                         impact of punctuate is typically not observable by other processing
> 
>                                         nodes
> 
>                                         in the topology.
> 
>                                         2. For controlling the emit frequency of downstream records.  Here,
> 
>                                         the
> 
>                                         punctuate is all about being observable by downstream processing
> 
>                                         nodes.
> 
>                                          
> 
>                                         A few releases back, we introduced record caches (DSL) and state
> 
>                                         store
> 
>                                         caches (Processor API) in KIP-63.  Here, we addressed a concern
> 
>                                         relating to
> 
>                                         (2) where some users needed to control -- here: limit -- the
> 
>                                         downstream
> 
>                                         output rate of Kafka Streams because the downstream systems/apps
> 
>                                         would not
> 
>                                         be able to keep up with the upstream output rate (Kafka scalability >
> 
>                                         their
> 
>                                         scalability).  The argument for KIP-63, which notably did not
> 
>                                         introduce a
> 
>                                         "trigger" API, was that such an interaction with downstream systems
> 
>                                         is an
> 
>                                         operational concern;  it should not impact the processing *logic* of
> 
>                                         your
> 
>                                         application, and thus we didn't want to complicate the Kafka Streams
> 
>                                         API,
> 
>                                         especially not the declarative DSL, with such operational concerns.
> 
>                                          
> 
>                                         This KIP's discussion on `punctuate()` takes us back in time (<--
> 
>                                         sorry, I
> 
>                                         couldn't resist to not make this pun :-P).  As a meta-comment, I am
> 
>                                         observing that our conversation is moving more and more into the
> 
>                                         direction
> 
>                                         of explicit "triggers" because, so far, I have seen only motivations
> 
>                                         for
> 
>                                         use cases in category (2), but none yet for (1)?  For example, some
> 
>                                         comments voiced here are about sth like "IF stream-time didn't
> 
>                                         trigger
> 
>                                         punctuate, THEN trigger punctuate based on processing-time".  Do we
> 
>                                         want
> 
>                                         this, and if so, for which use cases and benefits?  Also, on a
> 
>                                         related
> 
>                                         note, whatever we are discussing here will impact state store caches
> 
>                                         (Processor API) and perhaps also impact record caches (DSL), thus we
> 
>                                         should
> 
>                                         clarify any such impact here.
> 
>                                          
> 
>                                         Switching topics slightly.
> 
>                                          
> 
>                                         Jay wrote:
> 
>                                          
> 
>                                         One thing I've always found super important for this kind of design
> 
>                                         work
> 
>                                         is to do a really good job of cataloging the landscape of use cases
> 
>                                         and
> 
>                                         how prevalent each one is.
> 
>                                          
> 
>                                         +1 to this, as others have already said.
> 
>                                          
> 
>                                         Here, let me highlight -- just in case -- that when we talked about
> 
>                                         windowing use cases in the recent emails, the Processor API (where
> 
>                                         `punctuate` resides) does not have any notion of windowing at
> 
>                                         all.  If you
> 
>                                         want to do windowing *in the Processor API*, you must do so manually
> 
>                                         in
> 
>                                         combination with window stores.  For this reason I'd suggest to
> 
>                                         discuss use
> 
>                                         cases not just in general, but also in view of how you'd do so in the
> 
>                                         Processor API vs. in the DSL.  Right now, changing/improving
> 
>                                         `punctuate`
> 
>                                         does not impact the DSL at all, unless we add new functionality to
> 
>                                         it.
> 
>                                          
> 
>                                         Jay wrote in his strawman example:
> 
>                                          
> 
>                                         You aggregate click and impression data for a reddit like site.
> 
>                                         Every ten
> 
>                                         minutes you want to output a ranked list of the top 10 articles
> 
>                                         ranked by
> 
>                                         clicks/impressions for each geographical area. I want to be able
> 
>                                         run this
> 
>                                         in steady state as well as rerun to regenerate results (or catch up
> 
>                                         if it
> 
>                                         crashes).
> 
>                                          
> 
>                                         This is a good example for more than the obvious reason:  In KIP-63,
> 
>                                         we
> 
>                                         argued that the reason for saying "every ten minutes" above is not
> 
>                                         necessarily about because you want to output data *exactly* after ten
> 
>                                         minutes, but that you want to perform an aggregation based on 10-
> 
>                                         minute
> 
>                                         windows of input data; i.e., the point is about specifying the input
> 
>                                         for
> 
>                                         your aggregation, not or less about when the results of the
> 
>                                         aggregation
> 
>                                         should be send downstream.  To take an extreme example, you could
> 
>                                         disable
> 
>                                         record caches and let your app output a downstream update for every
> 
>                                         incoming input record.  If the last input record was from at minute 7
> 
>                                         of 10
> 
>                                         (for a 10-min window), then what your app would output at minute 10
> 
>                                         would
> 
>                                         be identical to what it had already emitted at minute 7 earlier
> 
>                                         anyways.
> 
>                                         This is particularly true when we take late-arriving data into
> 
>                                         account:  if
> 
>                                         a late record arrived at minute 13, your app would (by default) send
> 
>                                         a new
> 
>                                         update downstream, even though the "original" 10 minutes have already
> 
>                                         passed.
> 
>                                          
> 
>                                         Jay wrote...:
> 
>                                          
> 
>                                         There are a couple of tricky things that seem to make this hard
> 
>                                         with
> 
>                                          
> 
>                                         either
> 
>                                          
> 
>                                         of the options proposed:
> 
>                                         1. If I emit this data using event time I have the problem
> 
>                                         described where
> 
>                                         a geographical region with no new clicks or impressions will fail
> 
>                                         to
> 
>                                          
> 
>                                         output
> 
>                                          
> 
>                                         results.
> 
>                                          
> 
>                                         ...and Arun Mathew wrote:
> 
>                                          
> 
>                                          
> 
>                                         We window by the event time, but trigger punctuate in <punctuate
> 
>                                         interval>
> 
>                                         duration of system time, in the absence of an event crossing the
> 
>                                         punctuate
> 
>                                         event time.
> 
>                                          
> 
>                                         So, given what I wrote above about the status quo and what you can
> 
>                                         already
> 
>                                         do with it, is the concern that the state store cache doesn't give
> 
>                                         you
> 
>                                         *direct* control over "forcing an output after no later than X
> 
>                                         seconds [of
> 
>                                         processing-time]" but only indirect control through a cache
> 
>                                         size?  (Note
> 
>                                         that I am not dismissing the claims why this might be helpful.)
> 
>                                          
> 
>                                         Arun Mathew wrote:
> 
>                                          
> 
>                                         We are using Kafka Stream for our Audit Trail, where we need to
> 
>                                         output the
> 
>                                         event counts on each topic on each cluster aggregated over a 1
> 
>                                         minute
> 
>                                         window. We have to use event time to be able to cross check the
> 
>                                         counts.
> 
>                                          
> 
>                                         But
> 
>                                          
> 
>                                         we need to trigger punctuate [aggregate event pushes] by system
> 
>                                         time in
> 
>                                          
> 
>                                         the
> 
>                                          
> 
>                                         absence of events. Otherwise the event counts for unexpired windows
> 
>                                         would
> 
>                                         be 0 which is bad.
> 
>                                          
> 
>                                         Isn't the latter -- "count would be 0" -- the problem between the
> 
>                                         absence
> 
>                                         of output vs. an output of 0, similar to the use of `Option[T]` in
> 
>                                         Scala
> 
>                                         and the difference between `None` and `Some(0)`?  That is, isn't the
> 
>                                         root
> 
>                                         cause that the downstream system interprets the absence of output in
> 
>                                         a
> 
>                                         particular way ("No output after 1 minute = I consider the output to
> 
>                                         be
> 
>                                         0.")?  Arguably, you could also adapt the downstream system (if
> 
>                                         possible)
> 
>                                         to correctly handle the difference between absence of output vs.
> 
>                                         output of
> 
>                                         0.  I am not implying that we shouldn't care about such a use case,
> 
>                                         but
> 
>                                         want to understand the motivation better. :-)
> 
>                                          
> 
>                                         Also, to add some perspective, in some related discussions we talked
> 
>                                         about
> 
>                                         how a Kafka Streams application should not worry or not be coupled
> 
>                                         unnecessarily with such interpretation specifics in a downstream
> 
>                                         system's
> 
>                                         behavior.  After all, tomorrow your app's output might be consumed by
> 
>                                         more
> 
>                                         than just this one downstream system.  Arguably, Kafka Connect rather
> 
>                                         than
> 
>                                         Kafka Streams might be the best tool to link the universes of Kafka
> 
>                                         and
> 
>                                         downstream systems, including helping to reconcile the differences in
> 
>                                         how
> 
>                                         these systems interpret changes, updates, late-arriving data,
> 
>                                         etc.  Kafka
> 
>                                         Connect would allow you to decouple the Kafka Streams app's logical
> 
>                                         processing from the specifics of downstream systems, thanks to
> 
>                                         specific
> 
>                                         sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
> 
>                                         this
> 
>                                         decoupling with Kafka Connect help here?  (And if the answer is "Yes,
> 
>                                         but
> 
>                                         it's currently awkward to use Connect for this", this might be a
> 
>                                         problem we
> 
>                                         can solve, too.)
> 
>                                          
> 
>                                         Switching topics slightly again.
> 
>                                          
> 
>                                         Thomas wrote:
> 
>                                          
> 
>                                         I'm not entirely convinced that a separate callback (option C)
> 
>                                         is that messy (it could just be a default method with an empty
> 
>                                         implementation), but if we wanted a single API to handle both
> 
>                                         cases,
> 
>                                         how about something like the following?
> 
>                                          
> 
>                                         enum Time {
> 
>                                            STREAM,
> 
>                                            CLOCK
> 
>                                         }
> 
>                                          
> 
>                                         Yeah, I am on the fence here, too.  If we use the 1-method approach,
> 
>                                         then
> 
>                                         whatever the user is doing inside this method is a black box to Kafka
> 
>                                         Streams (similar to how we have no idea what the user does inside a
> 
>                                         `foreach` -- if the function passed to `foreach` writes to external
> 
>                                         systems, then Kafka Streams is totally unaware of the fact).  We
> 
>                                         won't
> 
>                                         know, for example, if the stream-time action has a smaller "trigger"
> 
>                                         frequency than the processing-time action.  Or, we won't know whether
> 
>                                         the
> 
>                                         user custom-codes a "not later than" trigger logic ("Do X every 1-
> 
>                                         minute of
> 
>                                         stream-time or 1-minute of processing-time, whichever comes
> 
>                                         first").  That
> 
>                                         said, I am not certain yet whether we would need such knowledge
> 
>                                         because,
> 
>                                         when using the Processor API, most of the work and decisions must be
> 
>                                         done
> 
>                                         by the user anyways.  It would matter though if the concept of
> 
>                                         "triggers"
> 
>                                         were to bubble up into the DSL because in the DSL the management of
> 
>                                         windowing, window stores, etc. must be done automatically by Kafka
> 
>                                         Streams.
> 
>                                          
> 
>                                         [In any case, btw, we have the corner case where the user configured
> 
>                                         the
> 
>                                         stream-time to be processing-time (e.g. via wall-clock timestamp
> 
>                                         extractor), at which point both punctuate variants are based on the
> 
>                                         same
> 
>                                         time semantics / timeline.]
> 
>                                          
> 
>                                         Again, I apologize for the wall of text.  Congratulations if you made
> 
>                                         it
> 
>                                         this far. :-)
> 
>                                          
> 
>                                         More than happy to hear your thoughts!
> 
>                                         Michael
> 
>                                          
> 
>                                         On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com>
>                                         <ma...@gmail.com> <ar...@gmail.com>
>                                         <ma...@gmail.com>
> 
>                                         wrote:
> 
>                                          
> 
>                                          
> 
>                                         Thanks Matthias.
> 
>                                         Sure, will correct it right away.
> 
>                                          
> 
>                                         On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io>
>                                         <ma...@confluent.io> <ma...@confluent.io>
>                                         <ma...@confluent.io>
> 
>                                         wrote:
> 
>                                          
> 
>                                         Thanks for preparing this page!
> 
>                                          
> 
>                                         About terminology:
> 
>                                          
> 
>                                         You introduce the term "event time" -- but we should call this
> 
>                                         "stream
> 
>                                         time" -- "stream time" is whatever TimestampExtractor returns and
> 
>                                         this
> 
>                                         could be event time, ingestion time, or processing/wall-clock time.
> 
>                                          
> 
>                                         Does this make sense to you?
> 
>                                          
> 
>                                          
> 
>                                          
> 
>                                         -Matthias
> 
>                                          
> 
>                                          
> 
>                                         On 4/10/17 4:58 AM, Arun Mathew wrote:
> 
>                                          
> 
>                                         Thanks Ewen.
> 
>                                          
> 
>                                         @Michal, @all, I have created a child page to start the Use Cases
> 
>                                          
> 
>                                         discussion [https://cwiki.apache.org/confluence/display/KAFKA/
> 
>                                         Punctuate+Use+Cases]. Please go through it and give your comments.
> 
>                                          
> 
>                                          
> 
>                                         @Tianji, Sorry for the delay. I am trying to make the patch
> 
>                                         public.
> 
>                                          
> 
>                                         --
> 
>                                         Arun Mathew
> 
>                                          
> 
>                                         On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io>
>                                         <ma...@confluent.io> <ew...@confluent.io>
>                                         <ma...@confluent.io>
> 
>                                         wrote:
> 
>                                          
> 
>                                             Arun,
> 
>                                          
> 
>                                             I've given you permission to edit the wiki. Let me know if
> 
>                                         you run
> 
>                                          
> 
>                                         into any
> 
>                                          
> 
>                                             issues.
> 
>                                          
> 
>                                             -Ewen
> 
>                                          
> 
>                                             On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp>
>                                         <ma...@yahoo-corp.jp> <am...@yahoo-corp.jp>
>                                         <ma...@yahoo-corp.jp>
> 
>                                          
> 
>                                         wrote:
> 
>                                          
> 
>                                          
> 
>                                             > Thanks Michal. I don’t have the access yet [arunmathew88].
> 
>                                         Should I
> 
>                                          
> 
>                                         be
> 
>                                          
> 
>                                             > sending a separate mail for this?
> 
>                                             >
> 
>                                             > I thought one of the person following this thread would be
> 
>                                         able to
> 
>                                          
> 
>                                         give me
> 
>                                          
> 
>                                             > access.
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > *From: *Michal Borowiecki <mi...@openbet.com>
>                                         <ma...@openbet.com> <mi...@openbet.com>
>                                         <ma...@openbet.com>
> 
>                                             > *Reply-To: *"dev@kafka.apache.org"
>                                         <ma...@kafka.apache.org> <de...@kafka.apache.org>
>                                         <ma...@kafka.apache.org> <de...@kafka.apache.org>
>                                         <ma...@kafka.apache.org> <de...@kafka.apache.org>
>                                         <ma...@kafka.apache.org>
> 
>                                             > *Date: *Friday, April 7, 2017 at 17:16
> 
>                                             > *To: *"dev@kafka.apache.org"
>                                         <ma...@kafka.apache.org> <de...@kafka.apache.org>
>                                         <ma...@kafka.apache.org> <de...@kafka.apache.org>
>                                         <ma...@kafka.apache.org> <de...@kafka.apache.org>
>                                         <ma...@kafka.apache.org>
> 
>                                             > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
> 
>                                         semantics
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > Hi Arun,
> 
>                                             >
> 
>                                             > I was thinking along the same lines as you, listing the use
> 
>                                         cases
> 
>                                          
> 
>                                         on the
> 
>                                          
> 
>                                             > wiki, but didn't find time to get around doing that yet.
> 
>                                             > Don't mind if you do it if you have access now.
> 
>                                             > I was thinking it would be nice if, once we have the use
> 
>                                         cases
> 
>                                          
> 
>                                         listed,
> 
>                                          
> 
>                                             > people could use likes to up-vote the use cases similar to
> 
>                                         what
> 
>                                          
> 
>                                         they're
> 
>                                          
> 
>                                             > working on.
> 
>                                             >
> 
>                                             > I should have a bit more time to action this in the next
> 
>                                         few days,
> 
>                                          
> 
>                                         but
> 
>                                          
> 
>                                             > happy for you to do it if you can beat me to it ;-)
> 
>                                             >
> 
>                                             > Cheers,
> 
>                                             > Michal
> 
>                                             >
> 
>                                             > On 07/04/17 04:39, Arun Mathew wrote:
> 
>                                             >
> 
>                                             > Sure, Thanks Matthias. My id is [arunmathew88].
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > Of course. I was thinking of a subpage where people can
> 
>                                          
> 
>                                         collaborate.
> 
>                                          
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > Will do as per Michael’s suggestion.
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > Regards,
> 
>                                             >
> 
>                                             > Arun Mathew
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io>
>                                         <ma...@confluent.io> <ma...@confluent.io>
>                                         <ma...@confluent.io>
> 
>                                         < 
> 
>                                          
> 
>                                         matthias@confluent.io
>                                         <ma...@confluent.io>> wrote:
> 
>                                          
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >     Please share your Wiki-ID and a committer can give you
> 
>                                         write
> 
>                                          
> 
>                                         access.
> 
>                                          
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >     Btw: as you did not initiate the KIP, you should not
> 
>                                         change the
> 
>                                          
> 
>                                         KIP
> 
>                                          
> 
>                                             >
> 
>                                             >     without the permission of the original author -- in
> 
>                                         this case
> 
>                                          
> 
>                                         Michael.
> 
>                                          
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >     So you might also just share your thought over the
> 
>                                         mailing list
> 
>                                          
> 
>                                         and
> 
>                                          
> 
>                                             >
> 
>                                             >     Michael can update the KIP page. Or, as an alternative,
> 
>                                         just
> 
>                                          
> 
>                                         create a
> 
>                                          
> 
>                                             >
> 
>                                             >     subpage for the KIP page.
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >     @Michael: WDYT?
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >     -Matthias
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >     On 4/6/17 8:05 PM, Arun Mathew wrote:
> 
>                                             >
> 
>                                             >     > Hi Jay,
> 
>                                             >
> 
>                                             >     >           Thanks for the advise, I would like to list
> 
>                                         down
> 
>                                          
> 
>                                         the use cases as
> 
>                                          
> 
>                                             >
> 
>                                             >     > per your suggestion. But it seems I don't have write
> 
>                                          
> 
>                                         permission to the
> 
>                                          
> 
>                                             >
> 
>                                             >     > Apache Kafka Confluent Space. Whom shall I request
> 
>                                         for it?
> 
>                                             >
> 
>                                             >     >
> 
>                                             >
> 
>                                             >     > Regarding your last question. We are using a patch in
> 
>                                         our
> 
>                                          
> 
>                                         production system
> 
>                                          
> 
>                                             >
> 
>                                             >     > which does exactly this.
> 
>                                             >
> 
>                                             >     > We window by the event time, but trigger punctuate in
> 
>                                          
> 
>                                         <punctuate interval>
> 
>                                          
> 
>                                             >
> 
>                                             >     > duration of system time, in the absence of an event
> 
>                                         crossing
> 
>                                          
> 
>                                         the punctuate
> 
>                                          
> 
>                                             >
> 
>                                             >     > event time.
> 
>                                             >
> 
>                                             >     >
> 
>                                             >
> 
>                                             >     > We are using Kafka Stream for our Audit Trail, where
> 
>                                         we need
> 
>                                          
> 
>                                         to output the
> 
>                                          
> 
>                                             >
> 
>                                             >     > event counts on each topic on each cluster aggregated
> 
>                                         over a
> 
>                                          
> 
>                                         1 minute
> 
>                                          
> 
>                                             >
> 
>                                             >     > window. We have to use event time to be able to cross
> 
>                                         check
> 
>                                          
> 
>                                         the counts. But
> 
>                                          
> 
>                                             >
> 
>                                             >     > we need to trigger punctuate [aggregate event pushes]
> 
>                                         by
> 
>                                          
> 
>                                         system time in the
> 
>                                          
> 
>                                             >
> 
>                                             >     > absence of events. Otherwise the event counts for
> 
>                                         unexpired
> 
>                                          
> 
>                                         windows would
> 
>                                          
> 
>                                             >
> 
>                                             >     > be 0 which is bad.
> 
>                                             >
> 
>                                             >     >
> 
>                                             >
> 
>                                             >     > "Maybe a hybrid solution works: I window by event
> 
>                                         time but
> 
>                                          
> 
>                                         trigger results
> 
>                                          
> 
>                                             >
> 
>                                             >     > by system time for windows that have updated? Not
> 
>                                         really sure
> 
>                                          
> 
>                                         the details
> 
>                                          
> 
>                                             >
> 
>                                             >     > of making that work. Does that work? Are there
> 
>                                         concrete
> 
>                                          
> 
>                                         examples where you
> 
>                                          
> 
>                                             >
> 
>                                             >     > actually want the current behavior?"
> 
>                                             >
> 
>                                             >     >
> 
>                                             >
> 
>                                             >     > --
> 
>                                             >
> 
>                                             >     > With Regards,
> 
>                                             >
> 
>                                             >     >
> 
>                                             >
> 
>                                             >     > Arun Mathew
> 
>                                             >
> 
>                                             >     > Yahoo! JAPAN Corporation
> 
>                                             >
> 
>                                             >     >
> 
>                                             >
> 
>                                             >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
> 
>                                          
> 
>                                         skyahead@gmail.com
>                                         <ma...@gmail.com>
>                                         <ma...@gmail.com> <sk...@gmail.com>
>                                         <ma...@gmail.com> wrote:
> 
>                                          
> 
>                                             >
> 
>                                             >     >
> 
>                                             >
> 
>                                             >     >> Hi Jay,
> 
>                                             >
> 
>                                             >     >>
> 
>                                             >
> 
>                                             >     >> The hybrid solution is exactly what I expect and
> 
>                                         need for
> 
>                                          
> 
>                                         our use cases
> 
>                                          
> 
>                                             >
> 
>                                             >     >> when dealing with telecom data.
> 
>                                             >
> 
>                                             >     >>
> 
>                                             >
> 
>                                             >     >> Thanks
> 
>                                             >
> 
>                                             >     >> Tianji
> 
>                                             >
> 
>                                             >     >>
> 
>                                             >
> 
>                                             >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
> 
>                                          
> 
>                                         jay@confluent.io
>                                         <ma...@confluent.io>
>                                         <ma...@confluent.io> <ja...@confluent.io>
>                                         <ma...@confluent.io> wrote:
> 
>                                          
> 
>                                             >
> 
>                                             >     >>
> 
>                                             >
> 
>                                             >     >>> Hey guys,
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>> One thing I've always found super important for
> 
>                                         this kind
> 
>                                          
> 
>                                         of design work
> 
>                                          
> 
>                                             >
> 
>                                             >     >> is
> 
>                                             >
> 
>                                             >     >>> to do a really good job of cataloging the landscape
> 
>                                         of use
> 
>                                          
> 
>                                         cases and how
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> prevalent each one is. By that I mean not just
> 
>                                         listing lots
> 
>                                          
> 
>                                         of uses, but
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> also grouping them into categories that
> 
>                                         functionally need
> 
>                                          
> 
>                                         the same thing.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> In the absence of this it is very hard to reason
> 
>                                         about
> 
>                                          
> 
>                                         design proposals.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> From the proposals so far I think we have a lot of
> 
>                                          
> 
>                                         discussion around
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> possible apis, but less around what the user needs
> 
>                                         for
> 
>                                          
> 
>                                         different use
> 
>                                          
> 
>                                             >
> 
>                                             >     >> cases
> 
>                                             >
> 
>                                             >     >>> and how they would implement that using the api.
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>> Here is an example:
> 
>                                             >
> 
>                                             >     >>> You aggregate click and impression data for a
> 
>                                         reddit like
> 
>                                          
> 
>                                         site. Every ten
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> minutes you want to output a ranked list of the top
> 
>                                         10
> 
>                                          
> 
>                                         articles ranked by
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> clicks/impressions for each geographical area. I
> 
>                                         want to be
> 
>                                          
> 
>                                         able run this
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> in steady state as well as rerun to regenerate
> 
>                                         results (or
> 
>                                          
> 
>                                         catch up if it
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> crashes).
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>> There are a couple of tricky things that seem to
> 
>                                         make this
> 
>                                          
> 
>                                         hard with
> 
>                                          
> 
>                                             >
> 
>                                             >     >> either
> 
>                                             >
> 
>                                             >     >>> of the options proposed:
> 
>                                             >
> 
>                                             >     >>> 1. If I emit this data using event time I have the
> 
>                                         problem
> 
>                                          
> 
>                                         described
> 
>                                          
> 
>                                             >
> 
>                                             >     >> where
> 
>                                             >
> 
>                                             >     >>> a geographical region with no new clicks or
> 
>                                         impressions
> 
>                                          
> 
>                                         will fail to
> 
>                                          
> 
>                                             >
> 
>                                             >     >> output
> 
>                                             >
> 
>                                             >     >>> results.
> 
>                                             >
> 
>                                             >     >>> 2. If I emit this data using system time I have the
> 
>                                         problem
> 
>                                          
> 
>                                         that when
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> reprocessing data my window may not be ten minutes
> 
>                                         but 10
> 
>                                          
> 
>                                         hours if my
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> processing is very fast so it dramatically changes
> 
>                                         the
> 
>                                          
> 
>                                         output.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>> Maybe a hybrid solution works: I window by event
> 
>                                         time but
> 
>                                          
> 
>                                         trigger results
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> by system time for windows that have updated? Not
> 
>                                         really
> 
>                                          
> 
>                                         sure the details
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> of making that work. Does that work? Are there
> 
>                                         concrete
> 
>                                          
> 
>                                         examples where
> 
>                                          
> 
>                                             >
> 
>                                             >     >> you
> 
>                                             >
> 
>                                             >     >>> actually want the current behavior?
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>> -Jay
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
> 
>                                          
> 
>                                         arunmathew88@gmail.com
>                                         <ma...@gmail.com>> <ar...@gmail.com>
>                                         <ma...@gmail.com> <ar...@gmail.com>
>                                         <ma...@gmail.com>
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> wrote:
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>>> Hi All,
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> Thanks for the KIP. We were also in need of a
> 
>                                         mechanism to
> 
>                                          
> 
>                                         trigger
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>> punctuate in the absence of events.
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> As I described in [
> 
>                                             >
> 
>                                             >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
> 
>                                             >
> 
>                                             >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
> 
>                                             >
> 
>                                             >     >>>> plugin.system.issuetabpanels:comment-
> 
>                                         tabpanel#comment-
> 
>                                          
> 
>                                         15926036
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>> ],
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>>    - Our approached involved using the event time
> 
>                                         by
> 
>                                          
> 
>                                         default.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>    - The method to check if there is any punctuate
> 
>                                         ready
> 
>                                          
> 
>                                         in the
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>    PunctuationQueue is triggered via the any event
> 
>                                          
> 
>                                         received by the
> 
>                                          
> 
>                                             >
> 
>                                             >     >> stream
> 
>                                             >
> 
>                                             >     >>>>    tread, or at the polling intervals in the
> 
>                                         absence of
> 
>                                          
> 
>                                         any events.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>    - When we create Punctuate objects (which
> 
>                                         contains the
> 
>                                          
> 
>                                         next event
> 
>                                          
> 
>                                             >
> 
>                                             >     >> time
> 
>                                             >
> 
>                                             >     >>>>    for punctuation and interval), we also record
> 
>                                         the
> 
>                                          
> 
>                                         creation time
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> (system
> 
>                                             >
> 
>                                             >     >>>>    time).
> 
>                                             >
> 
>                                             >     >>>>    - While checking for maturity of Punctuate
> 
>                                         Schedule by
> 
>                                             >
> 
>                                             >     >> mayBePunctuate
> 
>                                             >
> 
>                                             >     >>>>    method, we also check if the system clock has
> 
>                                         elapsed
> 
>                                          
> 
>                                         the punctuate
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>    interval since the schedule creation time.
> 
>                                             >
> 
>                                             >     >>>>    - In the absence of any event, or in the
> 
>                                         absence of any
> 
>                                          
> 
>                                         event for
> 
>                                          
> 
>                                             >
> 
>                                             >     >> one
> 
>                                             >
> 
>                                             >     >>>>    topic in the partition group assigned to the
> 
>                                         stream
> 
>                                          
> 
>                                         task, the system
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>> time
> 
>                                             >
> 
>                                             >     >>>>    will elapse the interval and we trigger a
> 
>                                         punctuate
> 
>                                          
> 
>                                         using the
> 
>                                          
> 
>                                             >
> 
>                                             >     >> expected
> 
>                                             >
> 
>                                             >     >>>>    punctuation event time.
> 
>                                             >
> 
>                                             >     >>>>    - we then create the next punctuation schedule
> 
>                                         as
> 
>                                          
> 
>                                         punctuation event
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> time
> 
>                                             >
> 
>                                             >     >>>>    + punctuation interval, [again recording the
> 
>                                         system
> 
>                                          
> 
>                                         time of creation
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> of
> 
>                                             >
> 
>                                             >     >>>> the
> 
>                                             >
> 
>                                             >     >>>>    schedule].
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> We call this a Hybrid Punctuate. Of course, this
> 
>                                         approach
> 
>                                          
> 
>                                         has pros and
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>> cons.
> 
>                                             >
> 
>                                             >     >>>> Pros
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>>    - Punctuates will happen in <punctuate
> 
>                                         interval> time
> 
>                                          
> 
>                                         duration at
> 
>                                          
> 
>                                             >
> 
>                                             >     >> max
> 
>                                             >
> 
>                                             >     >>> in
> 
>                                             >
> 
>                                             >     >>>>    terms of system time.
> 
>                                             >
> 
>                                             >     >>>>    - The semantics as a whole continues to revolve
> 
>                                         around
> 
>                                          
> 
>                                         event time.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>    - We can use the old data [old timestamps] to
> 
>                                         rerun any
> 
>                                          
> 
>                                         experiments
> 
>                                          
> 
>                                             >
> 
>                                             >     >> or
> 
>                                             >
> 
>                                             >     >>>>    tests.
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> Cons
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>>    - In case the  <punctuate interval> is not a
> 
>                                         time
> 
>                                          
> 
>                                         duration [say
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> logical
> 
>                                             >
> 
>                                             >     >>>>    time/event count], then the approach might not
> 
>                                         be
> 
>                                          
> 
>                                         meaningful.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>    - In case there is a case where we have to wait
> 
>                                         for an
> 
>                                          
> 
>                                         actual event
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> from
> 
>                                             >
> 
>                                             >     >>>>    a low event rate partition in the partition
> 
>                                         group, this
> 
>                                          
> 
>                                         approach
> 
>                                          
> 
>                                             >
> 
>                                             >     >> will
> 
>                                             >
> 
>                                             >     >>>> jump
> 
>                                             >
> 
>                                             >     >>>>    the gun.
> 
>                                             >
> 
>                                             >     >>>>    - in case the event processing cannot catch up
> 
>                                         with the
> 
>                                          
> 
>                                         event rate
> 
>                                          
> 
>                                             >
> 
>                                             >     >> and
> 
>                                             >
> 
>                                             >     >>>>    the expected timestamp events gets queued for
> 
>                                         long
> 
>                                          
> 
>                                         time, this
> 
>                                          
> 
>                                             >
> 
>                                             >     >> approach
> 
>                                             >
> 
>                                             >     >>>>    might jump the gun.
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> I believe the above approach and discussion goes
> 
>                                         close to
> 
>                                          
> 
>                                         the approach
> 
>                                          
> 
>                                             >
> 
>                                             >     >> A.
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> -----------
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> I like the idea of having an even count based
> 
>                                         punctuate.
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> -----------
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> I agree with the discussion around approach C,
> 
>                                         that we
> 
>                                          
> 
>                                         should provide
> 
>                                          
> 
>                                             >
> 
>                                             >     >> the
> 
>                                             >
> 
>                                             >     >>>> user with the option to choose system time or
> 
>                                         event time
> 
>                                          
> 
>                                         based
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> punctuates.
> 
>                                             >
> 
>                                             >     >>>> But I believe that the user predominantly wants to
> 
>                                         use
> 
>                                          
> 
>                                         event time while
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> not
> 
>                                             >
> 
>                                             >     >>>> missing out on regular punctuates due to event
> 
>                                         delays or
> 
>                                          
> 
>                                         event
> 
>                                          
> 
>                                             >
> 
>                                             >     >> absences.
> 
>                                             >
> 
>                                             >     >>>> Hence a complex punctuate option as Matthias
> 
>                                         mentioned
> 
>                                          
> 
>                                         (quoted below)
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> would
> 
>                                             >
> 
>                                             >     >>>> be most apt.
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> "- We might want to add "complex" schedules later
> 
>                                         on
> 
>                                          
> 
>                                         (like, punctuate
> 
>                                          
> 
>                                             >
> 
>                                             >     >> on
> 
>                                             >
> 
>                                             >     >>>> every 10 seconds event-time or 60 seconds system-
> 
>                                         time
> 
>                                          
> 
>                                         whatever comes
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>> first)."
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> -----------
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> I think I read somewhere that Kafka Streams
> 
>                                         started with
> 
>                                          
> 
>                                         System Time as
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> the
> 
>                                             >
> 
>                                             >     >>>> punctuation standard, but was later changed to
> 
>                                         Event Time.
> 
>                                          
> 
>                                         I guess
> 
>                                          
> 
>                                             >
> 
>                                             >     >> there
> 
>                                             >
> 
>                                             >     >>>> would be some good reason behind it. As Kafka
> 
>                                         Streams want
> 
>                                          
> 
>                                         to evolve
> 
>                                          
> 
>                                             >
> 
>                                             >     >> more
> 
>                                             >
> 
>                                             >     >>>> on the Stream Processing front, I believe the
> 
>                                         emphasis on
> 
>                                          
> 
>                                         event time
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> would
> 
>                                             >
> 
>                                             >     >>>> remain quite strong.
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> With Regards,
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> Arun Mathew
> 
>                                             >
> 
>                                             >     >>>> Yahoo! JAPAN Corporation, Tokyo
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
> 
>                                          
> 
>                                         tobecker@tivo.com
>                                         <ma...@tivo.com>> <to...@tivo.com>
>                                         <ma...@tivo.com> <to...@tivo.com>
>                                         <ma...@tivo.com>
> 
>                                          
> 
>                                             >
> 
>                                             >     >> wrote:
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>>> Yeah I like PuncutationType much better; I just
> 
>                                         threw
> 
>                                          
> 
>                                         Time out there
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>> more as a strawman than an actual suggestion ;) I
> 
>                                         still
> 
>                                          
> 
>                                         think it's
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>> worth considering what this buys us over an
> 
>                                         additional
> 
>                                          
> 
>                                         callback. I
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>> foresee a number of punctuate implementations
> 
>                                         following
> 
>                                          
> 
>                                         this pattern:
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>> public void punctuate(PunctuationType type) {
> 
>                                             >
> 
>                                             >     >>>>>     switch (type) {
> 
>                                             >
> 
>                                             >     >>>>>         case EVENT_TIME:
> 
>                                             >
> 
>                                             >     >>>>>             methodA();
> 
>                                             >
> 
>                                             >     >>>>>             break;
> 
>                                             >
> 
>                                             >     >>>>>         case SYSTEM_TIME:
> 
>                                             >
> 
>                                             >     >>>>>             methodB();
> 
>                                             >
> 
>                                             >     >>>>>             break;
> 
>                                             >
> 
>                                             >     >>>>>     }
> 
>                                             >
> 
>                                             >     >>>>> }
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>> I guess one advantage of this approach is we
> 
>                                         could add
> 
>                                          
> 
>                                         additional
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>> punctuation types later in a backwards compatible
> 
>                                         way
> 
>                                          
> 
>                                         (like event
> 
>                                          
> 
>                                             >
> 
>                                             >     >> count
> 
>                                             >
> 
>                                             >     >>>>> as you mentioned).
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>> -Tommy
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
> 
>                                         Sax wrote:
> 
>                                             >
> 
>                                             >     >>>>>> That sounds promising.
> 
>                                             >
> 
>                                             >     >>>>>>
> 
>                                             >
> 
>                                             >     >>>>>> I am just wondering if `Time` is the best name.
> 
>                                         Maybe we
> 
>                                          
> 
>                                         want to
> 
>                                          
> 
>                                             >
> 
>                                             >     >> add
> 
>                                             >
> 
>                                             >     >>>>>> other non-time based punctuations at some point
> 
>                                         later. I
> 
>                                          
> 
>                                         would
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>> suggest
> 
>                                             >
> 
>                                             >     >>>>>>
> 
>                                             >
> 
>                                             >     >>>>>> enum PunctuationType {
> 
>                                             >
> 
>                                             >     >>>>>>   EVENT_TIME,
> 
>                                             >
> 
>                                             >     >>>>>>   SYSTEM_TIME,
> 
>                                             >
> 
>                                             >     >>>>>> }
> 
>                                             >
> 
>                                             >     >>>>>>
> 
>                                             >
> 
>                                             >     >>>>>> or similar. Just to keep the door open -- it's
> 
>                                         easier to
> 
>                                          
> 
>                                         add new
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>> stuff
> 
>                                             >
> 
>                                             >     >>>>>> if the name is more generic.
> 
>                                             >
> 
>                                             >     >>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>
> 
>                                             >
> 
>                                             >     >>>>>> -Matthias
> 
>                                             >
> 
>                                             >     >>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>
> 
>                                             >
> 
>                                             >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>> I agree that the framework providing and
> 
>                                         managing the
> 
>                                          
> 
>                                         notion of
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> stream
> 
>                                             >
> 
>                                             >     >>>>>>> time is valuable and not something we would
> 
>                                         want to
> 
>                                          
> 
>                                         delegate to
> 
>                                          
> 
>                                             >
> 
>                                             >     >> the
> 
>                                             >
> 
>                                             >     >>>>>>> tasks. I'm not entirely convinced that a
> 
>                                         separate
> 
>                                          
> 
>                                         callback
> 
>                                          
> 
>                                             >
> 
>                                             >     >> (option
> 
>                                             >
> 
>                                             >     >>>>>>> C)
> 
>                                             >
> 
>                                             >     >>>>>>> is that messy (it could just be a default
> 
>                                         method with
> 
>                                          
> 
>                                         an empty
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> implementation), but if we wanted a single API
> 
>                                         to
> 
>                                          
> 
>                                         handle both
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> cases,
> 
>                                             >
> 
>                                             >     >>>>>>> how about something like the following?
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>> enum Time {
> 
>                                             >
> 
>                                             >     >>>>>>>    STREAM,
> 
>                                             >
> 
>                                             >     >>>>>>>    CLOCK
> 
>                                             >
> 
>                                             >     >>>>>>> }
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>> Then on ProcessorContext:
> 
>                                             >
> 
>                                             >     >>>>>>> context.schedule(Time time, long interval)  //
> 
>                                         We could
> 
>                                          
> 
>                                         allow
> 
>                                          
> 
>                                             >
> 
>                                             >     >> this
> 
>                                             >
> 
>                                             >     >>>>>>> to
> 
>                                             >
> 
>                                             >     >>>>>>> be called once for each value of time to mix
> 
>                                          
> 
>                                         approaches.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>> Then the Processor API becomes:
> 
>                                             >
> 
>                                             >     >>>>>>> punctuate(Time time) // time here denotes which
> 
>                                          
> 
>                                         schedule resulted
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> in
> 
>                                             >
> 
>                                             >     >>>>>>> this call.
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>> Thoughts?
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
> 
>                                         Sax
> 
>                                          
> 
>                                         wrote:
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> Thanks a lot for the KIP Michal,
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> I was thinking about the four options you
> 
>                                         proposed in
> 
>                                          
> 
>                                         more
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> details
> 
>                                             >
> 
>                                             >     >>>>>>>> and
> 
>                                             >
> 
>                                             >     >>>>>>>> this are my thoughts:
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> (A) You argue, that users can still
> 
>                                         "punctuate" on
> 
>                                          
> 
>                                         event-time
> 
>                                          
> 
>                                             >
> 
>                                             >     >> via
> 
>                                             >
> 
>                                             >     >>>>>>>> process(), but I am not sure if this is
> 
>                                         possible.
> 
>                                          
> 
>                                         Note, that
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> users
> 
>                                             >
> 
>                                             >     >>>>>>>> only
> 
>                                             >
> 
>                                             >     >>>>>>>> get record timestamps via context.timestamp().
> 
>                                         Thus,
> 
>                                          
> 
>                                         users
> 
>                                          
> 
>                                             >
> 
>                                             >     >> would
> 
>                                             >
> 
>                                             >     >>>>>>>> need
> 
>                                             >
> 
>                                             >     >>>>>>>> to
> 
>                                             >
> 
>                                             >     >>>>>>>> track the time progress per partition (based
> 
>                                         on the
> 
>                                          
> 
>                                         partitions
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> they
> 
>                                             >
> 
>                                             >     >>>>>>>> obverse via context.partition(). (This alone
> 
>                                         puts a
> 
>                                          
> 
>                                         huge burden
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> on
> 
>                                             >
> 
>                                             >     >>>>>>>> the
> 
>                                             >
> 
>                                             >     >>>>>>>> user by itself.) However, users are not
> 
>                                         notified at
> 
>                                          
> 
>                                         startup
> 
>                                          
> 
>                                             >
> 
>                                             >     >> what
> 
>                                             >
> 
>                                             >     >>>>>>>> partitions are assigned, and user are not
> 
>                                         notified
> 
>                                          
> 
>                                         when
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> partitions
> 
>                                             >
> 
>                                             >     >>>>>>>> get
> 
>                                             >
> 
>                                             >     >>>>>>>> revoked. Because this information is not
> 
>                                         available,
> 
>                                          
> 
>                                         it's not
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> possible
> 
>                                             >
> 
>                                             >     >>>>>>>> to
> 
>                                             >
> 
>                                             >     >>>>>>>> "manually advance" stream-time, and thus
> 
>                                         event-time
> 
>                                          
> 
>                                         punctuation
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> within
> 
>                                             >
> 
>                                             >     >>>>>>>> process() seems not to be possible -- or do
> 
>                                         you see a
> 
>                                          
> 
>                                         way to
> 
>                                          
> 
>                                             >
> 
>                                             >     >> get
> 
>                                             >
> 
>                                             >     >>>>>>>> it
> 
>                                             >
> 
>                                             >     >>>>>>>> done? And even if, it might still be too
> 
>                                         clumsy to
> 
>                                          
> 
>                                         use.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> (B) This does not allow to mix both
> 
>                                         approaches, thus
> 
>                                          
> 
>                                         limiting
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> what
> 
>                                             >
> 
>                                             >     >>>>>>>> users
> 
>                                             >
> 
>                                             >     >>>>>>>> can do.
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> (C) This should give all flexibility we need.
> 
>                                         However,
> 
>                                          
> 
>                                         just
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> adding
> 
>                                             >
> 
>                                             >     >>>>>>>> one
> 
>                                             >
> 
>                                             >     >>>>>>>> more method seems to be a solution that is too
> 
>                                         simple
> 
>                                          
> 
>                                         (cf my
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> comments
> 
>                                             >
> 
>                                             >     >>>>>>>> below).
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> (D) This might be hard to use. Also, I am not
> 
>                                         sure how
> 
>                                          
> 
>                                         a user
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> could
> 
>                                             >
> 
>                                             >     >>>>>>>> enable system-time and event-time punctuation
> 
>                                         in
> 
>                                          
> 
>                                         parallel.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> Overall options (C) seems to be the most
> 
>                                         promising
> 
>                                          
> 
>                                         approach to
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> me.
> 
>                                             >
> 
>                                             >     >>>>>>>> Because I also favor a clean API, we might
> 
>                                         keep
> 
>                                          
> 
>                                         current
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> punctuate()
> 
>                                             >
> 
>                                             >     >>>>>>>> as-is, but deprecate it -- so we can remove it
> 
>                                         at some
> 
>                                          
> 
>                                         later
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> point
> 
>                                             >
> 
>                                             >     >>>>>>>> when
> 
>                                             >
> 
>                                             >     >>>>>>>> people use the "new punctuate API".
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> Couple of follow up questions:
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> - I am wondering, if we should have two
> 
>                                         callback
> 
>                                          
> 
>                                         methods or
> 
>                                          
> 
>                                             >
> 
>                                             >     >> just
> 
>                                             >
> 
>                                             >     >>>>>>>> one
> 
>                                             >
> 
>                                             >     >>>>>>>> (ie, a unified for system and event time
> 
>                                         punctuation
> 
>                                          
> 
>                                         or one for
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> each?).
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> - If we have one, how can the user figure out,
> 
>                                         which
> 
>                                          
> 
>                                         condition
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> did
> 
>                                             >
> 
>                                             >     >>>>>>>> trigger?
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> - How would the API look like, for registering
> 
>                                          
> 
>                                         different
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> punctuate
> 
>                                             >
> 
>                                             >     >>>>>>>> schedules? The "type" must be somehow defined?
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> - We might want to add "complex" schedules
> 
>                                         later on
> 
>                                          
> 
>                                         (like,
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> punctuate
> 
>                                             >
> 
>                                             >     >>>>>>>> on
> 
>                                             >
> 
>                                             >     >>>>>>>> every 10 seconds event-time or 60 seconds
> 
>                                         system-time
> 
>                                          
> 
>                                         whatever
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> comes
> 
>                                             >
> 
>                                             >     >>>>>>>> first). I don't say we should add this right
> 
>                                         away, but
> 
>                                          
> 
>                                         we might
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> want
> 
>                                             >
> 
>                                             >     >>>>>>>> to
> 
>                                             >
> 
>                                             >     >>>>>>>> define the API in a way, that it allows
> 
>                                         extensions
> 
>                                          
> 
>                                         like this
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> later
> 
>                                             >
> 
>                                             >     >>>>>>>> on,
> 
>                                             >
> 
>                                             >     >>>>>>>> without redesigning the API (ie, the API
> 
>                                         should be
> 
>                                          
> 
>                                         designed
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> extensible)
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> - Did you ever consider count-based
> 
>                                         punctuation?
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> I understand, that you would like to solve a
> 
>                                         simple
> 
>                                          
> 
>                                         problem,
> 
>                                          
> 
>                                             >
> 
>                                             >     >> but
> 
>                                             >
> 
>                                             >     >>>>>>>> we
> 
>                                             >
> 
>                                             >     >>>>>>>> learned from the past, that just "adding some
> 
>                                         API"
> 
>                                          
> 
>                                         quickly
> 
>                                          
> 
>                                             >
> 
>                                             >     >> leads
> 
>                                             >
> 
>                                             >     >>>>>>>> to a
> 
>                                             >
> 
>                                             >     >>>>>>>> not very well defined API that needs time
> 
>                                         consuming
> 
>                                          
> 
>                                         clean up
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>> later on
> 
>                                             >
> 
>                                             >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
> 
>                                         holistic
> 
>                                             >
> 
>                                             >     >>>>>>>> punctuation
> 
>                                             >
> 
>                                             >     >>>>>>>> KIP
> 
>                                             >
> 
>                                             >     >>>>>>>> with this from the beginning on to avoid later
> 
>                                         painful
> 
>                                             >
> 
>                                             >     >> redesign.
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> -Matthias
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>> Thanks Thomas,
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>> I'm also wary of changing the existing
> 
>                                         semantics of
> 
>                                             >
> 
>                                             >     >> punctuate,
> 
>                                             >
> 
>                                             >     >>>>>>>>> for
> 
>                                             >
> 
>                                             >     >>>>>>>>> backward compatibility reasons, although I
> 
>                                         like the
> 
>                                             >
> 
>                                             >     >> conceptual
> 
>                                             >
> 
>                                             >     >>>>>>>>> simplicity of that option.
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>> Adding a new method to me feels safer but, in
> 
>                                         a way,
> 
>                                          
> 
>                                         uglier.
> 
>                                          
> 
>                                             >
> 
>                                             >     >> I
> 
>                                             >
> 
>                                             >     >>>>>>>>> added
> 
>                                             >
> 
>                                             >     >>>>>>>>> this to the KIP now as option (C).
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>> The TimestampExtractor mechanism is actually
> 
>                                         more
> 
>                                          
> 
>                                         flexible,
> 
>                                          
> 
>                                             >
> 
>                                             >     >> as
> 
>                                             >
> 
>                                             >     >>>>>>>>> it
> 
>                                             >
> 
>                                             >     >>>>>>>>> allows
> 
>                                             >
> 
>                                             >     >>>>>>>>> you to return any value, you're not limited
> 
>                                         to event
> 
>                                          
> 
>                                         time or
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>> system
> 
>                                             >
> 
>                                             >     >>>>>>>>> time
> 
>                                             >
> 
>                                             >     >>>>>>>>> (although I don't see an actual use case
> 
>                                         where you
> 
>                                          
> 
>                                         might need
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>> anything
> 
>                                             >
> 
>                                             >     >>>>>>>>> else then those two). Hence I also proposed
> 
>                                         the
> 
>                                          
> 
>                                         option to
> 
>                                          
> 
>                                             >
> 
>                                             >     >> allow
> 
>                                             >
> 
>                                             >     >>>>>>>>> users
> 
>                                             >
> 
>                                             >     >>>>>>>>> to, effectively, decide what "stream time" is
> 
>                                         for
> 
>                                          
> 
>                                         them given
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>> the
> 
>                                             >
> 
>                                             >     >>>>>>>>> presence or absence of messages, much like
> 
>                                         they can
> 
>                                          
> 
>                                         decide
> 
>                                          
> 
>                                             >
> 
>                                             >     >> what
> 
>                                             >
> 
>                                             >     >>>>>>>>> msg
> 
>                                             >
> 
>                                             >     >>>>>>>>> time
> 
>                                             >
> 
>                                             >     >>>>>>>>> means for them using the TimestampExtractor.
> 
>                                         What do
> 
>                                          
> 
>                                         you
> 
>                                          
> 
>                                             >
> 
>                                             >     >> think
> 
>                                             >
> 
>                                             >     >>>>>>>>> about
> 
>                                             >
> 
>                                             >     >>>>>>>>> that? This is probably most flexible but also
> 
>                                         most
> 
>                                             >
> 
>                                             >     >> complicated.
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>> All comments appreciated.
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>> Cheers,
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>> Michal
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
> 
>                                             >
> 
>                                             >     >>>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>> Although I fully agree we need a way to
> 
>                                         trigger
> 
>                                          
> 
>                                         periodic
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>> processing
> 
>                                             >
> 
>                                             >     >>>>>>>>>> that is independent from whether and when
> 
>                                         messages
> 
>                                          
> 
>                                         arrive,
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>> I'm
> 
>                                             >
> 
>                                             >     >>>>>>>>>> not sure
> 
>                                             >
> 
>                                             >     >>>>>>>>>> I like the idea of changing the existing
> 
>                                         semantics
> 
>                                          
> 
>                                         across
> 
>                                          
> 
>                                             >
> 
>                                             >     >> the
> 
>                                             >
> 
>                                             >     >>>>>>>>>> board.
> 
>                                             >
> 
>                                             >     >>>>>>>>>> What if we added an additional callback to
> 
>                                         Processor
> 
>                                          
> 
>                                         that
> 
>                                          
> 
>                                             >
> 
>                                             >     >> can
> 
>                                             >
> 
>                                             >     >>>>>>>>>> be
> 
>                                             >
> 
>                                             >     >>>>>>>>>> scheduled similarly to punctuate() but was
> 
>                                         always
> 
>                                          
> 
>                                         called at
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>> fixed, wall
> 
>                                             >
> 
>                                             >     >>>>>>>>>> clock based intervals? This way you wouldn't
> 
>                                         have to
> 
>                                          
> 
>                                         give
> 
>                                          
> 
>                                             >
> 
>                                             >     >> up
> 
>                                             >
> 
>                                             >     >>>>>>>>>> the
> 
>                                             >
> 
>                                             >     >>>>>>>>>> notion
> 
>                                             >
> 
>                                             >     >>>>>>>>>> of stream time to be able to do periodic
> 
>                                         processing.
> 
>                                             >
> 
>                                             >     >>>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
> 
>                                         Borowiecki
> 
>                                          
> 
>                                         wrote:
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> Hi all,
> 
>                                             >
> 
>                                             >     >>>>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> I have created a draft for KIP-138: Change
> 
>                                          
> 
>                                         punctuate
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> semantics
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> <https://cwiki.apache.org/
> 
>                                          
> 
>                                         confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
>                                         confluence/display/KAFKA/KIP->
>                                         <https://cwiki.apache.org/confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>                                         <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
> 
>                                          
> 
>                                             >
> 
>                                             >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI
>                                         P->
>                                         <https://cwiki.apache.org/confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>                                         <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
> 
>                                          
> 
>                                         138%
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> 3A+C
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> hange+
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> punctuate+semantics>
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> .
> 
>                                             >
> 
>                                             >     >>>>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> Appreciating there can be different views
> 
>                                         on
> 
>                                          
> 
>                                         system-time
> 
>                                          
> 
>                                             >
> 
>                                             >     >> vs
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> event-
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> time
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> semantics for punctuation depending on use-
> 
>                                         case and
> 
>                                          
> 
>                                         the
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> importance of
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> backwards compatibility of any such change,
> 
>                                         I've
> 
>                                          
> 
>                                         left it
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> quite
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> open
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> and
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> hope to fill in more info as the discussion
> 
>                                          
> 
>                                         progresses.
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> Thanks,
> 
>                                             >
> 
>                                             >     >>>>>>>>>>> Michal
> 
>                                             >
> 
>                                             >     >>>>>>> --
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>     Tommy Becker
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>     Senior Software Engineer
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>     tivo.com
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>> ________________________________
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>>>> This email and any attachments may contain
> 
>                                         confidential
> 
>                                          
> 
>                                         and
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> privileged material for the sole use of the
> 
>                                         intended
> 
>                                          
> 
>                                         recipient.
> 
>                                          
> 
>                                             >
> 
>                                             >     >> Any
> 
>                                             >
> 
>                                             >     >>>>>>> review, copying, or distribution of this email
> 
>                                         (or any
> 
>                                             >
> 
>                                             >     >> attachments)
> 
>                                             >
> 
>                                             >     >>>>>>> by others is prohibited. If you are not the
> 
>                                         intended
> 
>                                          
> 
>                                         recipient,
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> please contact the sender immediately and
> 
>                                         permanently
> 
>                                          
> 
>                                         delete this
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> email and any attachments. No employee or agent
> 
>                                         of TiVo
> 
>                                          
> 
>                                         Inc. is
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> authorized to conclude any binding agreement on
> 
>                                         behalf
> 
>                                          
> 
>                                         of TiVo
> 
>                                          
> 
>                                             >
> 
>                                             >     >> Inc.
> 
>                                             >
> 
>                                             >     >>>>>>> by email. Binding agreements with TiVo Inc. may
> 
>                                         only be
> 
>                                          
> 
>                                         made by a
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>>>> signed written agreement.
> 
>                                             >
> 
>                                             >     >>>>>>>
> 
>                                             >
> 
>                                             >     >>>>> --
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>>     Tommy Becker
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>>     Senior Software Engineer
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>>     tivo.com
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>> ________________________________
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>> This email and any attachments may contain
> 
>                                         confidential
> 
>                                          
> 
>                                         and
> 
>                                          
> 
>                                             >
> 
>                                             >     >> privileged
> 
>                                             >
> 
>                                             >     >>>>> material for the sole use of the intended
> 
>                                         recipient. Any
> 
>                                          
> 
>                                         review,
> 
>                                          
> 
>                                             >
> 
>                                             >     >>> copying,
> 
>                                             >
> 
>                                             >     >>>>> or distribution of this email (or any
> 
>                                         attachments) by
> 
>                                          
> 
>                                         others is
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>> prohibited.
> 
>                                             >
> 
>                                             >     >>>>> If you are not the intended recipient, please
> 
>                                         contact the
> 
>                                          
> 
>                                         sender
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>> immediately and permanently delete this email and
> 
>                                         any
> 
>                                          
> 
>                                         attachments. No
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>> employee or agent of TiVo Inc. is authorized to
> 
>                                         conclude
> 
>                                          
> 
>                                         any binding
> 
>                                          
> 
>                                             >
> 
>                                             >     >>>>> agreement on behalf of TiVo Inc. by email.
> 
>                                         Binding
> 
>                                          
> 
>                                         agreements with
> 
>                                          
> 
>                                             >
> 
>                                             >     >> TiVo
> 
>                                             >
> 
>                                             >     >>>>> Inc. may only be made by a signed written
> 
>                                         agreement.
> 
>                                             >
> 
>                                             >     >>>>>
> 
>                                             >
> 
>                                             >     >>>>
> 
>                                             >
> 
>                                             >     >>>
> 
>                                             >
> 
>                                             >     >>
> 
>                                             >
> 
>                                             >     >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > --
> 
>                                             >
> 
>                                             > <http://www.openbet.com/>
>                                         <http://www.openbet.com/> <http://www.openbet.com/>
>                                         <http://www.openbet.com/>
> 
>                                          
> 
>                                             >
> 
>                                             > *Michal Borowiecki*
> 
>                                             >
> 
>                                             > *Senior Software Engineer L4*
> 
>                                             >
> 
>                                             > *T: *
> 
>                                             >
> 
>                                             > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
> 
>                                             >
> 
>                                             > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > *E: *
> 
>                                             >
> 
>                                             > michal.borowiecki@openbet.com
>                                         <ma...@openbet.com>
> 
>                                             >
> 
>                                             > *W: *
> 
>                                             >
> 
>                                             > www.openbet.com
>                                         <http://www.openbet.com>
> 
>                                             >
> 
>                                             > *OpenBet Ltd*
> 
>                                             >
> 
>                                             > Chiswick Park Building 9
> 
>                                             >
> 
>                                             > 566 Chiswick High Rd
> 
>                                             >
> 
>                                             > London
> 
>                                             >
> 
>                                             > W4 5XT
> 
>                                             >
> 
>                                             > UK
> 
>                                             >
> 
>                                             > <https://www.openbet.com/email_promo>
>                                         <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>                                         <https://www.openbet.com/email_promo>
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                             > This message is confidential and intended only for the
> 
>                                         addressee.
> 
>                                          
> 
>                                         If you
> 
>                                          
> 
>                                             > have received this message in error, please immediately
> 
>                                         notify the
> 
>                                             > postmaster@openbet.com
>                                         <ma...@openbet.com> and delete it from your system as
> 
>                                         well as
> 
>                                          
> 
>                                         any
> 
>                                          
> 
>                                             > copies. The content of e-mails as well as traffic data may
> 
>                                         be
> 
>                                          
> 
>                                         monitored by
> 
>                                          
> 
>                                             > OpenBet for employment and security purposes. To protect
> 
>                                         the
> 
>                                          
> 
>                                         environment
> 
>                                          
> 
>                                             > please do not print this e-mail unless necessary. OpenBet
> 
>                                         Ltd.
> 
>                                          
> 
>                                         Registered
> 
>                                          
> 
>                                             > Office: Chiswick Park Building 9, 566 Chiswick High Road,
> 
>                                         London,
> 
>                                          
> 
>                                         W4 5XT,
> 
>                                          
> 
>                                             > United Kingdom. A company registered in England and Wales.
> 
>                                          
> 
>                                         Registered no.
> 
>                                          
> 
>                                             > 3134634. VAT no. GB927523612
> 
>                                             >
> 
>                                             >
> 
>                                             >
> 
>                                          
> 
>                                          
> 
>                                          
> 
>                                         --
> 
>                                          
> 
>                                          
> 
>                                             Tommy Becker
> 
>                                          
> 
>                                             Senior Software Engineer
> 
>                                          
> 
>                                             O +1 919.460.4747 <%28919%29%20460-4747>
> 
>                                          
> 
>                                          
> 
>                                             tivo.com
> 
>                                          
> 
>                                          
> 
>                                          
> 
>                                         ________________________________
> 
>                                          
> 
>                                         This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
> 
>                                          
> 
>                                          
> 
>                                         --
> 
>                                         <http://www.openbet.com/>
>                                         <http://www.openbet.com/> Michal Borowiecki
> 
>                                         Senior Software Engineer L4
> 
>                                         T: +44 208 742 1600 <+44%2020%208742%201600>
> 
>                                          
> 
>                                          
> 
>                                          
> 
>                             -- 
> 
>                             Signature
> 
>                             <http://www.openbet.com/>
>                             <http://www.openbet.com/>     Michal Borowiecki
> 
>                             Senior Software Engineer L4
> 
>                                 T:     +44 208 742 1600
> 
>                              
> 
>                                 
> 
>                                 +44 203 249 8448
> 
>                              
> 
>                                 
> 
>                                  
> 
>                                E:  michal.borowiecki@openbet.com
>                             <ma...@openbet.com>
> 
>                                W:  www.openbet.com <http://www.openbet.com> <http://www.openbet.com/>
>                             <http://www.openbet.com/>
> 
>                              
> 
>                                 
> 
>                                 OpenBet Ltd
> 
>                              
> 
>                                 Chiswick Park Building 9
> 
>                              
> 
>                                 566 Chiswick High Rd
> 
>                              
> 
>                                 London
> 
>                              
> 
>                                 W4 5XT
> 
>                              
> 
>                                 UK
> 
>                              
> 
>                                 
> 
>                             <https://www.openbet.com/email_promo>
>                             <https://www.openbet.com/email_promo>
> 
>                              
> 
>                             This message is confidential and intended only for the addressee. If
> 
>                             you have received this message in error, please immediately notify the
> 
>                             postmaster@openbet.com
>                             <ma...@openbet.com> <ma...@openbet.com>
>                             <ma...@openbet.com> and delete it
> 
>                             from your system as well as any copies. The content of e-mails as well
> 
>                             as traffic data may be monitored by OpenBet for employment and
> 
>                             security purposes. To protect the environment please do not print this
> 
>                             e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
> 
>                             Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
> 
>                             company registered in England and Wales. Registered no. 3134634. VAT
> 
>                             no. GB927523612
> 
>                              
> 
>                         -- 
> 
>                         Signature
> 
>                         <http://www.openbet.com/> <http://www.openbet.com/>   Michal Borowiecki
> 
>                         Senior Software Engineer L4
> 
>                           T:      +44 208 742 1600
> 
>                          
> 
>                           
> 
>                           +44 203 249 8448
> 
>                          
> 
>                           
> 
>                            
> 
>                           E:  michal.borowiecki@openbet.com
>                         <ma...@openbet.com>
> 
>                           W:   www.openbet.com <http://www.openbet.com> <http://www.openbet.com/> <http://www.openbet.com/>
> 
>                          
> 
>                           
> 
>                           OpenBet Ltd
> 
>                          
> 
>                           Chiswick Park Building 9
> 
>                          
> 
>                           566 Chiswick High Rd
> 
>                          
> 
>                           London
> 
>                          
> 
>                           W4 5XT
> 
>                          
> 
>                           UK
> 
>                          
> 
>                           
> 
>                         <https://www.openbet.com/email_promo>
>                         <https://www.openbet.com/email_promo>
> 
>                          
> 
>                         This message is confidential and intended only for the addressee. If you
> 
>                         have received this message in error, please immediately notify the
> 
>                         postmaster@openbet.com
>                         <ma...@openbet.com> <ma...@openbet.com>
>                         <ma...@openbet.com> and delete it
> 
>                         from your system as well as any copies. The content of e-mails as well
> 
>                         as traffic data may be monitored by OpenBet for employment and security
> 
>                         purposes. To protect the environment please do not print this e-mail
> 
>                         unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 
>                         9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> 
>                         registered in England and Wales. Registered no. 3134634. VAT no.
> 
>                         GB927523612
> 
>                          
> 
>                 -- 
> 
>                 Signature
> 
>                 <http://www.openbet.com/> <http://www.openbet.com/>    Michal Borowiecki
> 
>                 Senior Software Engineer L4
> 
>                        T:      +44 208 742 1600
> 
>                  
> 
>                        
> 
>                        +44 203 249 8448
> 
>                  
> 
>                        
> 
>                         
> 
>                        E:        michal.borowiecki@openbet.com
>                 <ma...@openbet.com>
> 
>                        W:      www.openbet.com <http://www.openbet.com> <http://www.openbet.com/> <http://www.openbet.com/>
> 
>                  
> 
>                        
> 
>                        OpenBet Ltd
> 
>                  
> 
>                        Chiswick Park Building 9
> 
>                  
> 
>                        566 Chiswick High Rd
> 
>                  
> 
>                        London
> 
>                  
> 
>                        W4 5XT
> 
>                  
> 
>                        UK
> 
>                  
> 
>                        
> 
>                 <https://www.openbet.com/email_promo>
>                 <https://www.openbet.com/email_promo>
> 
>                  
> 
>                 This message is confidential and intended only for the addressee. If you
> 
>                 have received this message in error, please immediately notify the
> 
>                 postmaster@openbet.com <ma...@openbet.com> <ma...@openbet.com>
>                 <ma...@openbet.com> and delete it
> 
>                 from your system as well as any copies. The content of e-mails as well
> 
>                 as traffic data may be monitored by OpenBet for employment and security
> 
>                 purposes. To protect the environment please do not print this e-mail
> 
>                 unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 
>                 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> 
>                 registered in England and Wales. Registered no. 3134634. VAT no.
> 
>                 GB927523612
> 
>                  
> 
>         -- 
> 
>         Signature
> 
>         <http://www.openbet.com/> <http://www.openbet.com/>  Michal Borowiecki
> 
>         Senior Software Engineer L4
> 
>            T:      +44 208 742 1600
> 
>          
> 
>            
> 
>            +44 203 249 8448
> 
>          
> 
>            
> 
>             
> 
>            E:      michal.borowiecki@openbet.com <ma...@openbet.com>
> 
>            W:      www.openbet.com <http://www.openbet.com> <http://www.openbet.com/> <http://www.openbet.com/>
> 
>          
> 
>            
> 
>            OpenBet Ltd
> 
>          
> 
>            Chiswick Park Building 9
> 
>          
> 
>            566 Chiswick High Rd
> 
>          
> 
>            London
> 
>          
> 
>            W4 5XT
> 
>          
> 
>            UK
> 
>          
> 
>            
> 
>         <https://www.openbet.com/email_promo>
>         <https://www.openbet.com/email_promo>
> 
>          
> 
>         This message is confidential and intended only for the addressee. If you
> 
>         have received this message in error, please immediately notify the
> 
>         postmaster@openbet.com <ma...@openbet.com> <ma...@openbet.com> <ma...@openbet.com> and delete it
> 
>         from your system as well as any copies. The content of e-mails as well
> 
>         as traffic data may be monitored by OpenBet for employment and security
> 
>         purposes. To protect the environment please do not print this e-mail
> 
>         unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 
>         9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> 
>         registered in England and Wales. Registered no. 3134634. VAT no.
> 
>         GB927523612
> 
>          
> 
>  
> 
> -- 
> 
> <http://www.openbet.com/>
> 
> 	
> 
> *Michal Borowiecki*
> 
> *Senior Software Engineer L4*
> 
> 	
> 
> *T: *
> 
> 	
> 
> +44 208 742 1600
> 
> 		
> 
> +44 203 249 8448
> 
> 		
> 
>  
> 
> 	
> 
> *E: *
> 
> 	
> 
> michal.borowiecki@openbet.com <ma...@openbet.com>
> 
> 	
> 
> *W: *
> 
> 	
> 
> www.openbet.com <http://www.openbet.com/>
> 
> 	
> 
> 	
> 
> *OpenBet Ltd*
> 
> 	
> 
> Chiswick Park Building 9
> 
> 	
> 
> 566 Chiswick High Rd
> 
> 	
> 
> London
> 
> 	
> 
> W4 5XT
> 
> 	
> 
> UK
> 
> 	
> 
> <https://www.openbet.com/email_promo>
> 
>  
> 
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com <ma...@openbet.com> and delete it
> from your system as well as any copies. The content of e-mails as well
> as traffic data may be monitored by OpenBet for employment and security
> purposes. To protect the environment please do not print this e-mail
> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> registered in England and Wales. Registered no. 3134634. VAT no. GB927523612
> 
>  
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Arun Mathew <am...@yahoo-corp.jp>.
Hi All,

Apologies for the delayed response.
As requested by Tianji, I am sharing our implementation of hybrid punctuate.

It is a rather small patch, a hack for our use-case I would say. I suggest you all to have a look.
You can view/(download) the diff from https://github.com/apache/kafka/compare/0.10.1.1...arunmathew88:hybrid-punctuate (.diff)

Now I will try to address some of matters below


> In the current stream-time punctuation scheme, the first PunctuationSchedule is set to 0 + interval (what else, since we haven't seen any records yet), but when triggered, it passes the current stream time as timestamp argument and uses that to schedule the next punctuation. It all makes sense.
> Arun, have you implemented any special handling for this case in your system?
In our implementation, we never send the system time as argument to punctuate, as it will be semantically incorrect to mix it. We use system time only to check if punctuate interval has elapsed in the wallclock. So in the situation of the 0 + interval as fist punctuation schedule, our hybrid punctuate implementation will call punctuate with (0+interval) as the argument, and (0 + interval  + interval) as the next punctuation schedule. Hence this flaw is not there.

However as I had mentioned in my original description mail (under the Cons section), and as pointed out by Matthias


>(2) IMHO the major disadvantage (issue?) of the hybrid approach is the
implicit assumption that even-time progresses at the same "speed" as
system-time during regular processing. This implies the assumption that
a slower progress in stream-time indicates the absence of input events

Though this can be tweaked to detect a slow stream time progression to block hybrid punctuation, it won’t look very clean.

@Michal, regarding 2b
> but I can't believe this to be the only reason behind this choice as minimum is not the only function to guarantee the group partition time never going back. Using the largest or the average among partitions' timestamp would also guaranteed the group timestamp not going back as timestamp never goes back for any individual partition.
So why was minimum chosen? Is it depended on by window semantics somewhere or anything else?

My thought as to why minimum was chosen can be illustrated via a simple use-case. Say we are consuming data from topic A event{ts, valueA} and topic B  event{ts, valueB}, and output to topic C event{ts, valueC}. If we have to output the valueC = valueA + valueB for matching ts values, then we need to punctuate to happen after all values for a specified ts has arrived. (of course we are skipping the case of missing events, late/out of order events etc).



> I agree that expressing the intended hybrid semantics is harder if we

offer only #StreamTime and #SystemTime punctuation. However, I also

believe that the hybrid approach is a "false friend" with regard to

reasoning about the semantics (it indicates that it more easy as it is

in reality). Therefore, we might be better off to not offer the hybrid

approach and make it clear to a developed, that it is hard to mix

#StreamTime and #SystemTime in a semantically sound way.

I agree with Matthias, that hybrid approach might not be a clean approach, it has its own assumptions, In particular I don’t like that the approach will have to virtually force push the stream time, for a low event rate partition in the partition group, to trigger a punctuate.  Nevertheless, I believe it is quite handy in a few use-cases.

I totally agree with the manthra of keeping things simple and straight forward at the platform level and let the developer build things on top of it. So let us think how to modify the punctuate semantics so that the user can implement desired effect, stream time /system time /hybrid punctuate at the developer level. I had to patch the Kafka Streams because I wasn’t able to implement it using the levers and switches provided by the KS library. I had originally just wanted a pure system time based punctuate, like the cron case discussed before.



It was a long read, since my last mail, I will go through all conversations again and update the Use-Cases document appropriately.

--
With Regards

Arun Mathew
Yahoo! JAPAN Corporation


From: Michal Borowiecki <mi...@openbet.com>
Reply-To: "dev@kafka.apache.org" <de...@kafka.apache.org>
Date: Tuesday, April 25, 2017 at 07:11
To: "dev@kafka.apache.org" <de...@kafka.apache.org>
Subject: Re: [DISCUSS] KIP-138: Change punctuate semantics


Sorry, my original point was lost or I didn't make it clear enough.

I don't challenge the need for a system-time interval upper bound, one we don't have now. On the contrary my own use-case depends on it. However, the hybrid does provide a system-time interval upper bound, even in the absence of records, it only fails to provide a lower bound.

As to heartbeats, IMO that wouldn't need a lower bound on the heartbeat interval, an upper bound should suffice.



Nevertheless, I think I found a flaw in the hybrid proposal, as follows:

In the current stream-time punctuation scheme, the first PunctuationSchedule is set to 0 + interval (what else, since we haven't seen any records yet), but when triggered, it passes the current stream time as timestamp argument and uses that to schedule the next punctuation. It all makes sense.

In the hybrid this would only work if the first punctuation is triggered by stream time advance but could have unintended consequences if the first triggering is by system time.

Imagine this scenario: my apps are down and I have loads of records to process (perhaps I reset offsets to reprocess data).

If I start my streams app but records don't start coming in immediately (perhaps there is another app upstream that I have yet to start), then the first punctuation will be triggered by system time. It won't have any stream-time to use so, the only timestamp it can use is the system-time timestamp, both as an argument to punctuate and for the next schedule.

But if then messages start flowing and have event time way in the past (assuming event/ingestion time extractor), then stream time will be behind system-time until the stream catches up (which may be never if I just want to run in historical data as an experiment/validation). However, punctuate (having been triggered first by system time) will keep on recurring at regular intervals and always passing in the system time (or to be precise: the system time of previous punctuation + interval). That is surely not what is desired or expected, given the steady inflow of records.

Arun, have you implemented any special handling for this case in your system?

As mentioned before the hybrid punctuation scheme is not easy to reason about, I can't deny that.



Happy to stick with the 2 PunctuationTypes proposal (stream time and system time). I'll try to write up how users can implement hybrid-like semantics themselves on top of that to address those demanding use-cases mentioned on the wiki.

Maybe it won't end up too complex after all, but as per the mantra, it's enough that it's made possible ;-)

Thanks,

Michal

On 24/04/17 18:22, Matthias J. Sax wrote:

Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?

If you go with stream-time and don't have any input records for all

partitions, punctuate would not be called at all, and thus your

dashboard would "freeze".



I thought about cron-type things, but aren't they better triggered by an

external scheduler (they're more flexible anyway), which then feeds

"commands" into the topology?

I guess it depends what kind of periodic action you want to trigger. The

"cron job" was just an analogy. Maybe it's just a heartbeat to some

other service, that signals that your Streams app is still running.





-Matthias





On 4/24/17 10:02 AM, Michal Borowiecki wrote:

Thanks!



Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?



Unless too frequent messages on replay could overpower it?





I thought about cron-type things, but aren't they better triggered by an

external scheduler (they're more flexible anyway), which then feeds

"commands" into the topology?



Just my 2c.



Cheers,



Michal





On 24/04/17 17:57, Matthias J. Sax wrote:

A simple example would be some dashboard app, that needs to get

"current" status in regular time intervals (ie, and real-time app).



Or something like a "scheduler" -- think "cron job" application.





-Matthias



On 4/24/17 2:23 AM, Michal Borowiecki wrote:

Hi Matthias,



I agree it's difficult to reason about the hybrid approach, I certainly

found it hard and I'm totally on board with the mantra.



I'd be happy to limit the scope of this KIP to add system-time

punctuation semantics (in addition to existing stream-time semantics)

and leave more complex schemes for users to implement on top of that.



Further additional PunctuationTypes, could then be added by future KIPs,

possibly including the hybrid approach once it has been given more thought.



There are real-time applications, that want to get

callbacks in regular system-time intervals (completely independent from

stream-time).

Can you please describe what they are, so that I can put them on the

wiki for later reference?



Thanks,



Michal





On 23/04/17 21:27, Matthias J. Sax wrote:

Hi,



I do like Damian's API proposal about the punctuation callback function.



I also did reread the KIP and thought about the semantics we want to

provide.



Given the above, I don't see a reason any more for a separate system-time based punctuation.

I disagree here. There are real-time applications, that want to get

callbacks in regular system-time intervals (completely independent from

stream-time). Thus we should allow this -- if we really follow the

"hybrid" approach, this could be configured with stream-time interval

infinite and delay whatever system-time punctuation interval you want to

have. However, I would like to add a proper API for this and do this

configuration under the hood (that would allow one implementation within

all kind of branching for different cases).



Thus, we definitely should have PunctutionType#StreamTime and

#SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not

a fan of your latest API proposal.





About the hybrid approach in general. On the one hand I like it, on the

other hand, it seems to be rather (1) complicated (not necessarily from

an implementation point of view, but for people to understand it) and

(2) mixes two semantics together in a "weird" way". Thus, I disagree with:



It may appear complicated at first but I do think these semantics will

still be more understandable to users than having 2 separate punctuation

schedules/callbacks with different PunctuationTypes.

This statement only holds if you apply strong assumptions that I don't

believe hold in general -- see (2) for details -- and I think it is

harder than you assume to reason about the hybrid approach in general.

IMHO, the hybrid approach is a "false friend" that seems to be easy to

reason about...





(1) Streams always embraced "easy to use" and we should really be

careful to keep it this way. On the other hand, as we are talking about

changes to PAPI, it won't affect DSL users (DSL does not use punctuation

at all at the moment), and thus, the "easy to use" mantra might not be

affected, while it will allow advanced users to express more complex stuff.



I like the mantra: "make simple thing easy and complex things possible".



(2) IMHO the major disadvantage (issue?) of the hybrid approach is the

implicit assumption that even-time progresses at the same "speed" as

system-time during regular processing. This implies the assumption that

a slower progress in stream-time indicates the absence of input events

(and that later arriving input events will have a larger event-time with

high probability). Even if this might be true for some use cases, I

doubt it holds in general. Assume that you get a spike in traffic and

for some reason stream-time does advance slowly because you have more

records to process. This might trigger a system-time based punctuation

call even if this seems not to be intended. I strongly believe that it

is not easy to reason about the semantics of the hybrid approach (even

if the intentional semantics would be super useful -- but I doubt that

we get want we ask for).



Thus, I also believe that one might need different "configuration"

values for the hybrid approach if you run the same code for different

scenarios: regular processing, re-processing, catching up scenario. And

as the term "configuration" implies, we might be better off to not mix

configuration with business logic that is expressed via code.





One more comment: I also don't think that the hybrid approach is

deterministic as claimed in the use-case subpage. I understand the

reasoning and agree, that it is deterministic if certain assumptions

hold -- compare above -- and if configured correctly. But strictly

speaking it's not because there is a dependency on system-time (and

IMHO, if system-time is involved it cannot be deterministic by definition).





I see how in theory this could be implemented on top of the 2 punctuate

callbacks with the 2 different PunctuationTypes (one stream-time based,

the other system-time based) but it would be a much more complicated

scheme and I don't want to suggest that.

I agree that expressing the intended hybrid semantics is harder if we

offer only #StreamTime and #SystemTime punctuation. However, I also

believe that the hybrid approach is a "false friend" with regard to

reasoning about the semantics (it indicates that it more easy as it is

in reality). Therefore, we might be better off to not offer the hybrid

approach and make it clear to a developed, that it is hard to mix

#StreamTime and #SystemTime in a semantically sound way.





Looking forward to your feedback. :)



-Matthias









On 4/22/17 11:43 AM, Michal Borowiecki wrote:

Hi all,



Looking for feedback on the functional interface approach Damian

proposed. What do people think?



Further on the semantics of triggering punctuate though:



I ran through the 2 use cases that Arun had kindly put on the wiki

(https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)

in my head and on a whiteboard and I can't find a better solution than

the "hybrid" approach he had proposed.



I see how in theory this could be implemented on top of the 2 punctuate

callbacks with the 2 different PunctuationTypes (one stream-time based,

the other system-time based) but it would be a much more complicated

scheme and I don't want to suggest that.



However, to add to the hybrid algorithm proposed, I suggest one

parameter to that: a tolerance period, expressed in milliseconds

system-time, after which the punctuation will be invoked in case the

stream-time advance hasn't triggered it within the requested interval

since the last invocation of punctuate (as recorded in system-time)



This would allow a user-defined tolerance for late arriving events. The

trade off would be left for the user to decide: regular punctuation in

the case of absence of events vs allowing for records arriving late or

some build-up due to processing not catching up with the event rate.

In the one extreme, this tolerance could be set to infinity, turning

hybrid into simply stream-time based punctuate, like we have now. In the

other extreme, the tolerance could be set to 0, resulting in a

system-time upper bound on the effective punctuation interval.



Given the above, I don't see a reason any more for a separate

system-time based punctuation. The "hybrid" approach with 0ms tolerance

would under normal operation trigger at regular intervals wrt the

system-time, except in cases of re-play/catch-up, where the stream time

advances faster than system time. In these cases punctuate would happen

more often than the specified interval wrt system time. However, the

use-cases that need system-time punctuations (that I've seen at least)

really only have a need for an upper bound on punctuation delay but

don't need a lower bound.



To that effect I'd propose the api to be as follows, on ProcessorContext:



schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)



schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate



Punctuation is triggered when either:

- the stream time advances past the (stream time of the previous

punctuation) + interval;

- or (iff the toleranceInterval is set) when the system time advances

past the (system time of the previous punctuation) + interval +

toleranceInterval



In either case:

- we trigger punctuate passing as the argument the stream time at which

the current punctuation was meant to happen

- next punctuate is scheduled at (stream time at which the current

punctuation was meant to happen) + interval



It may appear complicated at first but I do think these semantics will

still be more understandable to users than having 2 separate punctuation

schedules/callbacks with different PunctuationTypes.







PS. Having re-read this, maybe the following alternative would be easier

to understand (WDYT?):



schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval



schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate



Punctuation is triggered when either:

- the stream time advances past the (stream time of the previous

punctuation) + streamTimeInterval;

- or (iff systemTimeUpperBound is set) when the system time advances

past the (system time of the previous punctuation) + systemTimeUpperBound



Awaiting comments.



Thanks,

Michal



On 21/04/17 16:56, Michal Borowiecki wrote:

Yes, that's what I meant. Just wanted to highlight we'd deprecate it

in favour of something that doesn't return a record. Not a problem though.





On 21/04/17 16:32, Damian Guy wrote:

Thanks Michal,

I agree Transformer.punctuate should also be void, but we can deprecate

that too in favor of the new interface.



Thanks for the javadoc PR!



Cheers,

Damian



On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <

michal.borowiecki@openbet.com<ma...@openbet.com>> wrote:



Yes, that looks better to me.



Note that punctuate on Transformer is currently returning a record, but I

think it's ok to have all output records be sent via

ProcessorContext.forward, which has to be used anyway if you want to send

multiple records from one invocation of punctuate.



This way it's consistent between Processor and Transformer.





BTW, looking at this I found a glitch in the javadoc and put a comment

there:



https://github.com/apache/kafka/pull/2413/files#r112634612



and PR: https://github.com/apache/kafka/pull/2884



Cheers,



Michal

On 20/04/17 18:55, Damian Guy wrote:



Hi Michal,



Thanks for the KIP. I'd like to propose a bit more of a radical change to

the API.

1. deprecate the punctuate method on Processor

2. create a new Functional Interface just for Punctuation, something like:

interface Punctuator {

    void punctuate(long timestamp)

}

3. add a new schedule function to ProcessorContext: schedule(long

interval, PunctuationType type, Punctuator callback)

4. deprecate the existing schedule function



Thoughts?



Thanks,

Damian



On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <

michal.borowiecki@openbet.com<ma...@openbet.com>> wrote:



Hi Thomas,



I would say our use cases fall in the same category as yours.



1) One is expiry of old records, it's virtually identical to yours.



2) Second one is somewhat more convoluted but boils down to the same type

of design:



Incoming messages carry a number of fields, including a timestamp.



Outgoing messages contain derived fields, one of them (X) is depended on

by the timestamp input field (Y) and some other input field (Z).



Since the output field X is derived in some non-trivial way, we don't

want to force the logic onto downstream apps. Instead we want to calculate

it in the Kafka Streams app, which means we re-calculate X as soon as the

timestamp in Y is reached (wall clock time) and send a message if it

changed (I say "if" because the derived field (X) is also conditional on

another input field Z).



So we have kv stores with the records and an additional kv store with

timestamp->id mapping which act like an index where we periodically do a

ranged query.



Initially we naively tried doing it in punctuate which of course didn't

work when there were no regular msgs on the input topic.

Since this was before 0.10.1 and state stores weren't query-able from

outside we created a "ticker" that produced msgs once per second onto

another topic and fed it into the same topology to trigger punctuate.

This didn't work either, which was much more surprising to us at the

time, because it was not obvious at all that punctuate is only triggered if

*all* input partitions receive messages regularly.

In the end we had to break this into 2 separate Kafka Streams. Main

transformer doesn't use punctuate but sends values of timestamp field Y and

the id to a "scheduler" topic where also the periodic ticks are sent. This

is consumed by the second topology and is its only input topic. There's a

transformer on that topic which populates and updates the time-based

indexes and polls them from punctuate. If the time in the timestamp

elapsed, the record id is sent to the main transformer, which

updates/deletes the record from the main kv store and forwards the

transformed record to the output topic.



To me this setup feels horrendously complicated for what it does.



We could incrementally improve on this since 0.10.1 to poll the

timestamp->id "index" stores from some code outside the KafkaStreams

topology so that at least we wouldn't need the extra topic for "ticks".

However, the ticks don't feel so hacky when you realise they give you

some hypothetical benefits in predictability. You can reprocess the

messages in a reproducible manner, since the topologies use event-time,

just that the event time is simply the wall-clock time fed into a topic by

the ticks. (NB in our use case we haven't yet found a need for this kind of

reprocessing).

To make that work though, we would have to have the stream time advance

based on the presence of msgs on the "tick" topic, regardless of the

presence of messages on the other input topic.



Same as in the expiry use case, both the wall-clock triggered punctuate

and the hybrid would work to simplify this a lot.



3) Finally, I have a 3rd use case in the making but I'm still looking if

we can achieve it using session windows instead. I'll keep you posted if we

have to go with punctuate there too.



Thanks,

Michal





On 11/04/17 20:52, Thomas Becker wrote:



Here's an example that we currently have.  We have a streams processor

that does a transform from one topic into another. One of the fields in

the source topic record is an expiration time, and one of the functions

of the processor is to ensure that expired records get deleted promptly

after that time passes (typically days or weeks after the message was

originally produced). To do that, the processor keeps a state store of

keys and expiration times, iterates that store in punctuate(), and

emits delete (null) records for expired items. This needs to happen at

some minimum interval regardless of the incoming message rate of the

source topic.



In this scenario, the expiration of records is the primary function of

punctuate, and therefore the key requirement is that the wall-clock

measured time between punctuate calls have some upper-bound. So a pure

wall-clock based schedule would be fine for our needs. But the proposed

"hybrid" system would also be acceptable if that satisfies a broader

range of use-cases.



On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:



I apologize for the longer email below.  To my defense, it started

out much

shorter. :-)  Also, to be super-clear, I am intentionally playing

devil's

advocate for a number of arguments brought forth in order to help

improve

this KIP -- I am not implying I necessarily disagree with the

arguments.



That aside, here are some further thoughts.



First, there are (at least?) two categories for actions/behavior you

invoke

via punctuate():



1. For internal housekeeping of your Processor or Transformer (e.g.,

to

periodically commit to a custom store, to do metrics/logging).  Here,

the

impact of punctuate is typically not observable by other processing

nodes

in the topology.

2. For controlling the emit frequency of downstream records.  Here,

the

punctuate is all about being observable by downstream processing

nodes.



A few releases back, we introduced record caches (DSL) and state

store

caches (Processor API) in KIP-63.  Here, we addressed a concern

relating to

(2) where some users needed to control -- here: limit -- the

downstream

output rate of Kafka Streams because the downstream systems/apps

would not

be able to keep up with the upstream output rate (Kafka scalability >

their

scalability).  The argument for KIP-63, which notably did not

introduce a

"trigger" API, was that such an interaction with downstream systems

is an

operational concern;  it should not impact the processing *logic* of

your

application, and thus we didn't want to complicate the Kafka Streams

API,

especially not the declarative DSL, with such operational concerns.



This KIP's discussion on `punctuate()` takes us back in time (<--

sorry, I

couldn't resist to not make this pun :-P).  As a meta-comment, I am

observing that our conversation is moving more and more into the

direction

of explicit "triggers" because, so far, I have seen only motivations

for

use cases in category (2), but none yet for (1)?  For example, some

comments voiced here are about sth like "IF stream-time didn't

trigger

punctuate, THEN trigger punctuate based on processing-time".  Do we

want

this, and if so, for which use cases and benefits?  Also, on a

related

note, whatever we are discussing here will impact state store caches

(Processor API) and perhaps also impact record caches (DSL), thus we

should

clarify any such impact here.



Switching topics slightly.



Jay wrote:



One thing I've always found super important for this kind of design

work

is to do a really good job of cataloging the landscape of use cases

and

how prevalent each one is.



+1 to this, as others have already said.



Here, let me highlight -- just in case -- that when we talked about

windowing use cases in the recent emails, the Processor API (where

`punctuate` resides) does not have any notion of windowing at

all.  If you

want to do windowing *in the Processor API*, you must do so manually

in

combination with window stores.  For this reason I'd suggest to

discuss use

cases not just in general, but also in view of how you'd do so in the

Processor API vs. in the DSL.  Right now, changing/improving

`punctuate`

does not impact the DSL at all, unless we add new functionality to

it.



Jay wrote in his strawman example:



You aggregate click and impression data for a reddit like site.

Every ten

minutes you want to output a ranked list of the top 10 articles

ranked by

clicks/impressions for each geographical area. I want to be able

run this

in steady state as well as rerun to regenerate results (or catch up

if it

crashes).



This is a good example for more than the obvious reason:  In KIP-63,

we

argued that the reason for saying "every ten minutes" above is not

necessarily about because you want to output data *exactly* after ten

minutes, but that you want to perform an aggregation based on 10-

minute

windows of input data; i.e., the point is about specifying the input

for

your aggregation, not or less about when the results of the

aggregation

should be send downstream.  To take an extreme example, you could

disable

record caches and let your app output a downstream update for every

incoming input record.  If the last input record was from at minute 7

of 10

(for a 10-min window), then what your app would output at minute 10

would

be identical to what it had already emitted at minute 7 earlier

anyways.

This is particularly true when we take late-arriving data into

account:  if

a late record arrived at minute 13, your app would (by default) send

a new

update downstream, even though the "original" 10 minutes have already

passed.



Jay wrote...:



There are a couple of tricky things that seem to make this hard

with



either



of the options proposed:

1. If I emit this data using event time I have the problem

described where

a geographical region with no new clicks or impressions will fail

to



output



results.



...and Arun Mathew wrote:





We window by the event time, but trigger punctuate in <punctuate

interval>

duration of system time, in the absence of an event crossing the

punctuate

event time.



So, given what I wrote above about the status quo and what you can

already

do with it, is the concern that the state store cache doesn't give

you

*direct* control over "forcing an output after no later than X

seconds [of

processing-time]" but only indirect control through a cache

size?  (Note

that I am not dismissing the claims why this might be helpful.)



Arun Mathew wrote:



We are using Kafka Stream for our Audit Trail, where we need to

output the

event counts on each topic on each cluster aggregated over a 1

minute

window. We have to use event time to be able to cross check the

counts.



But



we need to trigger punctuate [aggregate event pushes] by system

time in



the



absence of events. Otherwise the event counts for unexpired windows

would

be 0 which is bad.



Isn't the latter -- "count would be 0" -- the problem between the

absence

of output vs. an output of 0, similar to the use of `Option[T]` in

Scala

and the difference between `None` and `Some(0)`?  That is, isn't the

root

cause that the downstream system interprets the absence of output in

a

particular way ("No output after 1 minute = I consider the output to

be

0.")?  Arguably, you could also adapt the downstream system (if

possible)

to correctly handle the difference between absence of output vs.

output of

0.  I am not implying that we shouldn't care about such a use case,

but

want to understand the motivation better. :-)



Also, to add some perspective, in some related discussions we talked

about

how a Kafka Streams application should not worry or not be coupled

unnecessarily with such interpretation specifics in a downstream

system's

behavior.  After all, tomorrow your app's output might be consumed by

more

than just this one downstream system.  Arguably, Kafka Connect rather

than

Kafka Streams might be the best tool to link the universes of Kafka

and

downstream systems, including helping to reconcile the differences in

how

these systems interpret changes, updates, late-arriving data,

etc.  Kafka

Connect would allow you to decouple the Kafka Streams app's logical

processing from the specifics of downstream systems, thanks to

specific

sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would

this

decoupling with Kafka Connect help here?  (And if the answer is "Yes,

but

it's currently awkward to use Connect for this", this might be a

problem we

can solve, too.)



Switching topics slightly again.



Thomas wrote:



I'm not entirely convinced that a separate callback (option C)

is that messy (it could just be a default method with an empty

implementation), but if we wanted a single API to handle both

cases,

how about something like the following?



enum Time {

   STREAM,

   CLOCK

}



Yeah, I am on the fence here, too.  If we use the 1-method approach,

then

whatever the user is doing inside this method is a black box to Kafka

Streams (similar to how we have no idea what the user does inside a

`foreach` -- if the function passed to `foreach` writes to external

systems, then Kafka Streams is totally unaware of the fact).  We

won't

know, for example, if the stream-time action has a smaller "trigger"

frequency than the processing-time action.  Or, we won't know whether

the

user custom-codes a "not later than" trigger logic ("Do X every 1-

minute of

stream-time or 1-minute of processing-time, whichever comes

first").  That

said, I am not certain yet whether we would need such knowledge

because,

when using the Processor API, most of the work and decisions must be

done

by the user anyways.  It would matter though if the concept of

"triggers"

were to bubble up into the DSL because in the DSL the management of

windowing, window stores, etc. must be done automatically by Kafka

Streams.



[In any case, btw, we have the corner case where the user configured

the

stream-time to be processing-time (e.g. via wall-clock timestamp

extractor), at which point both punctuate variants are based on the

same

time semantics / timeline.]



Again, I apologize for the wall of text.  Congratulations if you made

it

this far. :-)



More than happy to hear your thoughts!

Michael



On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>

wrote:





Thanks Matthias.

Sure, will correct it right away.



On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>

wrote:



Thanks for preparing this page!



About terminology:



You introduce the term "event time" -- but we should call this

"stream

time" -- "stream time" is whatever TimestampExtractor returns and

this

could be event time, ingestion time, or processing/wall-clock time.



Does this make sense to you?







-Matthias





On 4/10/17 4:58 AM, Arun Mathew wrote:



Thanks Ewen.



@Michal, @all, I have created a child page to start the Use Cases



discussion [https://cwiki.apache.org/confluence/display/KAFKA/

Punctuate+Use+Cases]. Please go through it and give your comments.





@Tianji, Sorry for the delay. I am trying to make the patch

public.



--

Arun Mathew



On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>

wrote:



    Arun,



    I've given you permission to edit the wiki. Let me know if

you run



into any



    issues.



    -Ewen



    On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp><ma...@yahoo-corp.jp> <am...@yahoo-corp.jp>



wrote:





    > Thanks Michal. I don’t have the access yet [arunmathew88].

Should I



be



    > sending a separate mail for this?

    >

    > I thought one of the person following this thread would be

able to



give me



    > access.

    >

    >

    >

    > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>

    > *Reply-To: *"dev@kafka.apache.org"<ma...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>

    > *Date: *Friday, April 7, 2017 at 17:16

    > *To: *"dev@kafka.apache.org"<ma...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>

    > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate

semantics

    >

    >

    >

    > Hi Arun,

    >

    > I was thinking along the same lines as you, listing the use

cases



on the



    > wiki, but didn't find time to get around doing that yet.

    > Don't mind if you do it if you have access now.

    > I was thinking it would be nice if, once we have the use

cases



listed,



    > people could use likes to up-vote the use cases similar to

what



they're



    > working on.

    >

    > I should have a bit more time to action this in the next

few days,



but



    > happy for you to do it if you can beat me to it ;-)

    >

    > Cheers,

    > Michal

    >

    > On 07/04/17 04:39, Arun Mathew wrote:

    >

    > Sure, Thanks Matthias. My id is [arunmathew88].

    >

    >

    >

    > Of course. I was thinking of a subpage where people can



collaborate.



    >

    >

    >

    > Will do as per Michael’s suggestion.

    >

    >

    >

    > Regards,

    >

    > Arun Mathew

    >

    >

    >

    > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>

<



matthias@confluent.io<ma...@confluent.io>> wrote:



    >

    >

    >

    >     Please share your Wiki-ID and a committer can give you

write



access.



    >

    >

    >

    >     Btw: as you did not initiate the KIP, you should not

change the



KIP



    >

    >     without the permission of the original author -- in

this case



Michael.



    >

    >

    >

    >     So you might also just share your thought over the

mailing list



and



    >

    >     Michael can update the KIP page. Or, as an alternative,

just



create a



    >

    >     subpage for the KIP page.

    >

    >

    >

    >     @Michael: WDYT?

    >

    >

    >

    >

    >

    >     -Matthias

    >

    >

    >

    >

    >

    >     On 4/6/17 8:05 PM, Arun Mathew wrote:

    >

    >     > Hi Jay,

    >

    >     >           Thanks for the advise, I would like to list

down



the use cases as



    >

    >     > per your suggestion. But it seems I don't have write



permission to the



    >

    >     > Apache Kafka Confluent Space. Whom shall I request

for it?

    >

    >     >

    >

    >     > Regarding your last question. We are using a patch in

our



production system



    >

    >     > which does exactly this.

    >

    >     > We window by the event time, but trigger punctuate in



<punctuate interval>



    >

    >     > duration of system time, in the absence of an event

crossing



the punctuate



    >

    >     > event time.

    >

    >     >

    >

    >     > We are using Kafka Stream for our Audit Trail, where

we need



to output the



    >

    >     > event counts on each topic on each cluster aggregated

over a



1 minute



    >

    >     > window. We have to use event time to be able to cross

check



the counts. But



    >

    >     > we need to trigger punctuate [aggregate event pushes]

by



system time in the



    >

    >     > absence of events. Otherwise the event counts for

unexpired



windows would



    >

    >     > be 0 which is bad.

    >

    >     >

    >

    >     > "Maybe a hybrid solution works: I window by event

time but



trigger results



    >

    >     > by system time for windows that have updated? Not

really sure



the details



    >

    >     > of making that work. Does that work? Are there

concrete



examples where you



    >

    >     > actually want the current behavior?"

    >

    >     >

    >

    >     > --

    >

    >     > With Regards,

    >

    >     >

    >

    >     > Arun Mathew

    >

    >     > Yahoo! JAPAN Corporation

    >

    >     >

    >

    >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <



skyahead@gmail.com<ma...@gmail.com> <sk...@gmail.com> wrote:



    >

    >     >

    >

    >     >> Hi Jay,

    >

    >     >>

    >

    >     >> The hybrid solution is exactly what I expect and

need for



our use cases



    >

    >     >> when dealing with telecom data.

    >

    >     >>

    >

    >     >> Thanks

    >

    >     >> Tianji

    >

    >     >>

    >

    >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <



jay@confluent.io<ma...@confluent.io> <ja...@confluent.io> wrote:



    >

    >     >>

    >

    >     >>> Hey guys,

    >

    >     >>>

    >

    >     >>> One thing I've always found super important for

this kind



of design work



    >

    >     >> is

    >

    >     >>> to do a really good job of cataloging the landscape

of use



cases and how



    >

    >     >>> prevalent each one is. By that I mean not just

listing lots



of uses, but



    >

    >     >>> also grouping them into categories that

functionally need



the same thing.



    >

    >     >>> In the absence of this it is very hard to reason

about



design proposals.



    >

    >     >>> From the proposals so far I think we have a lot of



discussion around



    >

    >     >>> possible apis, but less around what the user needs

for



different use



    >

    >     >> cases

    >

    >     >>> and how they would implement that using the api.

    >

    >     >>>

    >

    >     >>> Here is an example:

    >

    >     >>> You aggregate click and impression data for a

reddit like



site. Every ten



    >

    >     >>> minutes you want to output a ranked list of the top

10



articles ranked by



    >

    >     >>> clicks/impressions for each geographical area. I

want to be



able run this



    >

    >     >>> in steady state as well as rerun to regenerate

results (or



catch up if it



    >

    >     >>> crashes).

    >

    >     >>>

    >

    >     >>> There are a couple of tricky things that seem to

make this



hard with



    >

    >     >> either

    >

    >     >>> of the options proposed:

    >

    >     >>> 1. If I emit this data using event time I have the

problem



described



    >

    >     >> where

    >

    >     >>> a geographical region with no new clicks or

impressions



will fail to



    >

    >     >> output

    >

    >     >>> results.

    >

    >     >>> 2. If I emit this data using system time I have the

problem



that when



    >

    >     >>> reprocessing data my window may not be ten minutes

but 10



hours if my



    >

    >     >>> processing is very fast so it dramatically changes

the



output.



    >

    >     >>>

    >

    >     >>> Maybe a hybrid solution works: I window by event

time but



trigger results



    >

    >     >>> by system time for windows that have updated? Not

really



sure the details



    >

    >     >>> of making that work. Does that work? Are there

concrete



examples where



    >

    >     >> you

    >

    >     >>> actually want the current behavior?

    >

    >     >>>

    >

    >     >>> -Jay

    >

    >     >>>

    >

    >     >>>

    >

    >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <



arunmathew88@gmail.com<ma...@gmail.com>> <ar...@gmail.com> <ar...@gmail.com>



    >

    >     >>> wrote:

    >

    >     >>>

    >

    >     >>>> Hi All,

    >

    >     >>>>

    >

    >     >>>> Thanks for the KIP. We were also in need of a

mechanism to



trigger



    >

    >     >>>> punctuate in the absence of events.

    >

    >     >>>>

    >

    >     >>>> As I described in [

    >

    >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?

    >

    >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.

    >

    >     >>>> plugin.system.issuetabpanels:comment-

tabpanel#comment-



15926036



    >

    >     >>>> ],

    >

    >     >>>>

    >

    >     >>>>    - Our approached involved using the event time

by



default.



    >

    >     >>>>    - The method to check if there is any punctuate

ready



in the



    >

    >     >>>>    PunctuationQueue is triggered via the any event



received by the



    >

    >     >> stream

    >

    >     >>>>    tread, or at the polling intervals in the

absence of



any events.



    >

    >     >>>>    - When we create Punctuate objects (which

contains the



next event



    >

    >     >> time

    >

    >     >>>>    for punctuation and interval), we also record

the



creation time



    >

    >     >>> (system

    >

    >     >>>>    time).

    >

    >     >>>>    - While checking for maturity of Punctuate

Schedule by

    >

    >     >> mayBePunctuate

    >

    >     >>>>    method, we also check if the system clock has

elapsed



the punctuate



    >

    >     >>>>    interval since the schedule creation time.

    >

    >     >>>>    - In the absence of any event, or in the

absence of any



event for



    >

    >     >> one

    >

    >     >>>>    topic in the partition group assigned to the

stream



task, the system



    >

    >     >>>> time

    >

    >     >>>>    will elapse the interval and we trigger a

punctuate



using the



    >

    >     >> expected

    >

    >     >>>>    punctuation event time.

    >

    >     >>>>    - we then create the next punctuation schedule

as



punctuation event



    >

    >     >>> time

    >

    >     >>>>    + punctuation interval, [again recording the

system



time of creation



    >

    >     >>> of

    >

    >     >>>> the

    >

    >     >>>>    schedule].

    >

    >     >>>>

    >

    >     >>>> We call this a Hybrid Punctuate. Of course, this

approach



has pros and



    >

    >     >>>> cons.

    >

    >     >>>> Pros

    >

    >     >>>>

    >

    >     >>>>    - Punctuates will happen in <punctuate

interval> time



duration at



    >

    >     >> max

    >

    >     >>> in

    >

    >     >>>>    terms of system time.

    >

    >     >>>>    - The semantics as a whole continues to revolve

around



event time.



    >

    >     >>>>    - We can use the old data [old timestamps] to

rerun any



experiments



    >

    >     >> or

    >

    >     >>>>    tests.

    >

    >     >>>>

    >

    >     >>>> Cons

    >

    >     >>>>

    >

    >     >>>>    - In case the  <punctuate interval> is not a

time



duration [say



    >

    >     >>> logical

    >

    >     >>>>    time/event count], then the approach might not

be



meaningful.



    >

    >     >>>>    - In case there is a case where we have to wait

for an



actual event



    >

    >     >>> from

    >

    >     >>>>    a low event rate partition in the partition

group, this



approach



    >

    >     >> will

    >

    >     >>>> jump

    >

    >     >>>>    the gun.

    >

    >     >>>>    - in case the event processing cannot catch up

with the



event rate



    >

    >     >> and

    >

    >     >>>>    the expected timestamp events gets queued for

long



time, this



    >

    >     >> approach

    >

    >     >>>>    might jump the gun.

    >

    >     >>>>

    >

    >     >>>> I believe the above approach and discussion goes

close to



the approach



    >

    >     >> A.

    >

    >     >>>>

    >

    >     >>>> -----------

    >

    >     >>>>

    >

    >     >>>> I like the idea of having an even count based

punctuate.

    >

    >     >>>>

    >

    >     >>>> -----------

    >

    >     >>>>

    >

    >     >>>> I agree with the discussion around approach C,

that we



should provide



    >

    >     >> the

    >

    >     >>>> user with the option to choose system time or

event time



based



    >

    >     >>> punctuates.

    >

    >     >>>> But I believe that the user predominantly wants to

use



event time while



    >

    >     >>> not

    >

    >     >>>> missing out on regular punctuates due to event

delays or



event



    >

    >     >> absences.

    >

    >     >>>> Hence a complex punctuate option as Matthias

mentioned



(quoted below)



    >

    >     >>> would

    >

    >     >>>> be most apt.

    >

    >     >>>>

    >

    >     >>>> "- We might want to add "complex" schedules later

on



(like, punctuate



    >

    >     >> on

    >

    >     >>>> every 10 seconds event-time or 60 seconds system-

time



whatever comes



    >

    >     >>>> first)."

    >

    >     >>>>

    >

    >     >>>> -----------

    >

    >     >>>>

    >

    >     >>>> I think I read somewhere that Kafka Streams

started with



System Time as



    >

    >     >>> the

    >

    >     >>>> punctuation standard, but was later changed to

Event Time.



I guess



    >

    >     >> there

    >

    >     >>>> would be some good reason behind it. As Kafka

Streams want



to evolve



    >

    >     >> more

    >

    >     >>>> on the Stream Processing front, I believe the

emphasis on



event time



    >

    >     >>> would

    >

    >     >>>> remain quite strong.

    >

    >     >>>>

    >

    >     >>>>

    >

    >     >>>> With Regards,

    >

    >     >>>>

    >

    >     >>>> Arun Mathew

    >

    >     >>>> Yahoo! JAPAN Corporation, Tokyo

    >

    >     >>>>

    >

    >     >>>>

    >

    >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <



tobecker@tivo.com<ma...@tivo.com>> <to...@tivo.com> <to...@tivo.com>



    >

    >     >> wrote:

    >

    >     >>>>

    >

    >     >>>>> Yeah I like PuncutationType much better; I just

threw



Time out there



    >

    >     >>>>> more as a strawman than an actual suggestion ;) I

still



think it's



    >

    >     >>>>> worth considering what this buys us over an

additional



callback. I



    >

    >     >>>>> foresee a number of punctuate implementations

following



this pattern:



    >

    >     >>>>>

    >

    >     >>>>> public void punctuate(PunctuationType type) {

    >

    >     >>>>>     switch (type) {

    >

    >     >>>>>         case EVENT_TIME:

    >

    >     >>>>>             methodA();

    >

    >     >>>>>             break;

    >

    >     >>>>>         case SYSTEM_TIME:

    >

    >     >>>>>             methodB();

    >

    >     >>>>>             break;

    >

    >     >>>>>     }

    >

    >     >>>>> }

    >

    >     >>>>>

    >

    >     >>>>> I guess one advantage of this approach is we

could add



additional



    >

    >     >>>>> punctuation types later in a backwards compatible

way



(like event



    >

    >     >> count

    >

    >     >>>>> as you mentioned).

    >

    >     >>>>>

    >

    >     >>>>> -Tommy

    >

    >     >>>>>

    >

    >     >>>>>

    >

    >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.

Sax wrote:

    >

    >     >>>>>> That sounds promising.

    >

    >     >>>>>>

    >

    >     >>>>>> I am just wondering if `Time` is the best name.

Maybe we



want to



    >

    >     >> add

    >

    >     >>>>>> other non-time based punctuations at some point

later. I



would



    >

    >     >>>>>> suggest

    >

    >     >>>>>>

    >

    >     >>>>>> enum PunctuationType {

    >

    >     >>>>>>   EVENT_TIME,

    >

    >     >>>>>>   SYSTEM_TIME,

    >

    >     >>>>>> }

    >

    >     >>>>>>

    >

    >     >>>>>> or similar. Just to keep the door open -- it's

easier to



add new



    >

    >     >>>>>> stuff

    >

    >     >>>>>> if the name is more generic.

    >

    >     >>>>>>

    >

    >     >>>>>>

    >

    >     >>>>>> -Matthias

    >

    >     >>>>>>

    >

    >     >>>>>>

    >

    >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:

    >

    >     >>>>>>>

    >

    >     >>>>>>> I agree that the framework providing and

managing the



notion of



    >

    >     >>>>>>> stream

    >

    >     >>>>>>> time is valuable and not something we would

want to



delegate to



    >

    >     >> the

    >

    >     >>>>>>> tasks. I'm not entirely convinced that a

separate



callback



    >

    >     >> (option

    >

    >     >>>>>>> C)

    >

    >     >>>>>>> is that messy (it could just be a default

method with



an empty



    >

    >     >>>>>>> implementation), but if we wanted a single API

to



handle both



    >

    >     >>>>>>> cases,

    >

    >     >>>>>>> how about something like the following?

    >

    >     >>>>>>>

    >

    >     >>>>>>> enum Time {

    >

    >     >>>>>>>    STREAM,

    >

    >     >>>>>>>    CLOCK

    >

    >     >>>>>>> }

    >

    >     >>>>>>>

    >

    >     >>>>>>> Then on ProcessorContext:

    >

    >     >>>>>>> context.schedule(Time time, long interval)  //

We could



allow



    >

    >     >> this

    >

    >     >>>>>>> to

    >

    >     >>>>>>> be called once for each value of time to mix



approaches.



    >

    >     >>>>>>>

    >

    >     >>>>>>> Then the Processor API becomes:

    >

    >     >>>>>>> punctuate(Time time) // time here denotes which



schedule resulted



    >

    >     >>>>>>> in

    >

    >     >>>>>>> this call.

    >

    >     >>>>>>>

    >

    >     >>>>>>> Thoughts?

    >

    >     >>>>>>>

    >

    >     >>>>>>>

    >

    >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.

Sax



wrote:



    >

    >     >>>>>>>>

    >

    >     >>>>>>>> Thanks a lot for the KIP Michal,

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> I was thinking about the four options you

proposed in



more



    >

    >     >>>>>>>> details

    >

    >     >>>>>>>> and

    >

    >     >>>>>>>> this are my thoughts:

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> (A) You argue, that users can still

"punctuate" on



event-time



    >

    >     >> via

    >

    >     >>>>>>>> process(), but I am not sure if this is

possible.



Note, that



    >

    >     >>>>>>>> users

    >

    >     >>>>>>>> only

    >

    >     >>>>>>>> get record timestamps via context.timestamp().

Thus,



users



    >

    >     >> would

    >

    >     >>>>>>>> need

    >

    >     >>>>>>>> to

    >

    >     >>>>>>>> track the time progress per partition (based

on the



partitions



    >

    >     >>>>>>>> they

    >

    >     >>>>>>>> obverse via context.partition(). (This alone

puts a



huge burden



    >

    >     >>>>>>>> on

    >

    >     >>>>>>>> the

    >

    >     >>>>>>>> user by itself.) However, users are not

notified at



startup



    >

    >     >> what

    >

    >     >>>>>>>> partitions are assigned, and user are not

notified



when



    >

    >     >>>>>>>> partitions

    >

    >     >>>>>>>> get

    >

    >     >>>>>>>> revoked. Because this information is not

available,



it's not



    >

    >     >>>>>>>> possible

    >

    >     >>>>>>>> to

    >

    >     >>>>>>>> "manually advance" stream-time, and thus

event-time



punctuation



    >

    >     >>>>>>>> within

    >

    >     >>>>>>>> process() seems not to be possible -- or do

you see a



way to



    >

    >     >> get

    >

    >     >>>>>>>> it

    >

    >     >>>>>>>> done? And even if, it might still be too

clumsy to



use.



    >

    >     >>>>>>>>

    >

    >     >>>>>>>> (B) This does not allow to mix both

approaches, thus



limiting



    >

    >     >>>>>>>> what

    >

    >     >>>>>>>> users

    >

    >     >>>>>>>> can do.

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> (C) This should give all flexibility we need.

However,



just



    >

    >     >>>>>>>> adding

    >

    >     >>>>>>>> one

    >

    >     >>>>>>>> more method seems to be a solution that is too

simple



(cf my



    >

    >     >>>>>>>> comments

    >

    >     >>>>>>>> below).

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> (D) This might be hard to use. Also, I am not

sure how



a user



    >

    >     >>>>>>>> could

    >

    >     >>>>>>>> enable system-time and event-time punctuation

in



parallel.



    >

    >     >>>>>>>>

    >

    >     >>>>>>>>

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> Overall options (C) seems to be the most

promising



approach to



    >

    >     >>>>>>>> me.

    >

    >     >>>>>>>> Because I also favor a clean API, we might

keep



current



    >

    >     >>>>>>>> punctuate()

    >

    >     >>>>>>>> as-is, but deprecate it -- so we can remove it

at some



later



    >

    >     >>>>>>>> point

    >

    >     >>>>>>>> when

    >

    >     >>>>>>>> people use the "new punctuate API".

    >

    >     >>>>>>>>

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> Couple of follow up questions:

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> - I am wondering, if we should have two

callback



methods or



    >

    >     >> just

    >

    >     >>>>>>>> one

    >

    >     >>>>>>>> (ie, a unified for system and event time

punctuation



or one for



    >

    >     >>>>>>>> each?).

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> - If we have one, how can the user figure out,

which



condition



    >

    >     >>>>>>>> did

    >

    >     >>>>>>>> trigger?

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> - How would the API look like, for registering



different



    >

    >     >>>>>>>> punctuate

    >

    >     >>>>>>>> schedules? The "type" must be somehow defined?

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> - We might want to add "complex" schedules

later on



(like,



    >

    >     >>>>>>>> punctuate

    >

    >     >>>>>>>> on

    >

    >     >>>>>>>> every 10 seconds event-time or 60 seconds

system-time



whatever



    >

    >     >>>>>>>> comes

    >

    >     >>>>>>>> first). I don't say we should add this right

away, but



we might



    >

    >     >>>>>>>> want

    >

    >     >>>>>>>> to

    >

    >     >>>>>>>> define the API in a way, that it allows

extensions



like this



    >

    >     >>>>>>>> later

    >

    >     >>>>>>>> on,

    >

    >     >>>>>>>> without redesigning the API (ie, the API

should be



designed



    >

    >     >>>>>>>> extensible)

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> - Did you ever consider count-based

punctuation?

    >

    >     >>>>>>>>

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> I understand, that you would like to solve a

simple



problem,



    >

    >     >> but

    >

    >     >>>>>>>> we

    >

    >     >>>>>>>> learned from the past, that just "adding some

API"



quickly



    >

    >     >> leads

    >

    >     >>>>>>>> to a

    >

    >     >>>>>>>> not very well defined API that needs time

consuming



clean up



    >

    >     >>>>>>>> later on

    >

    >     >>>>>>>> via other KIPs. Thus, I would prefer to get a

holistic

    >

    >     >>>>>>>> punctuation

    >

    >     >>>>>>>> KIP

    >

    >     >>>>>>>> with this from the beginning on to avoid later

painful

    >

    >     >> redesign.

    >

    >     >>>>>>>>

    >

    >     >>>>>>>>

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> -Matthias

    >

    >     >>>>>>>>

    >

    >     >>>>>>>>

    >

    >     >>>>>>>>

    >

    >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>> Thanks Thomas,

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>> I'm also wary of changing the existing

semantics of

    >

    >     >> punctuate,

    >

    >     >>>>>>>>> for

    >

    >     >>>>>>>>> backward compatibility reasons, although I

like the

    >

    >     >> conceptual

    >

    >     >>>>>>>>> simplicity of that option.

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>> Adding a new method to me feels safer but, in

a way,



uglier.



    >

    >     >> I

    >

    >     >>>>>>>>> added

    >

    >     >>>>>>>>> this to the KIP now as option (C).

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>> The TimestampExtractor mechanism is actually

more



flexible,



    >

    >     >> as

    >

    >     >>>>>>>>> it

    >

    >     >>>>>>>>> allows

    >

    >     >>>>>>>>> you to return any value, you're not limited

to event



time or



    >

    >     >>>>>>>>> system

    >

    >     >>>>>>>>> time

    >

    >     >>>>>>>>> (although I don't see an actual use case

where you



might need



    >

    >     >>>>>>>>> anything

    >

    >     >>>>>>>>> else then those two). Hence I also proposed

the



option to



    >

    >     >> allow

    >

    >     >>>>>>>>> users

    >

    >     >>>>>>>>> to, effectively, decide what "stream time" is

for



them given



    >

    >     >>>>>>>>> the

    >

    >     >>>>>>>>> presence or absence of messages, much like

they can



decide



    >

    >     >> what

    >

    >     >>>>>>>>> msg

    >

    >     >>>>>>>>> time

    >

    >     >>>>>>>>> means for them using the TimestampExtractor.

What do



you



    >

    >     >> think

    >

    >     >>>>>>>>> about

    >

    >     >>>>>>>>> that? This is probably most flexible but also

most

    >

    >     >> complicated.

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>> All comments appreciated.

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>> Cheers,

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>> Michal

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>>

    >

    >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:

    >

    >     >>>>>>>>>>

    >

    >     >>>>>>>>>>

    >

    >     >>>>>>>>>> Although I fully agree we need a way to

trigger



periodic



    >

    >     >>>>>>>>>> processing

    >

    >     >>>>>>>>>> that is independent from whether and when

messages



arrive,



    >

    >     >>>>>>>>>> I'm

    >

    >     >>>>>>>>>> not sure

    >

    >     >>>>>>>>>> I like the idea of changing the existing

semantics



across



    >

    >     >> the

    >

    >     >>>>>>>>>> board.

    >

    >     >>>>>>>>>> What if we added an additional callback to

Processor



that



    >

    >     >> can

    >

    >     >>>>>>>>>> be

    >

    >     >>>>>>>>>> scheduled similarly to punctuate() but was

always



called at



    >

    >     >>>>>>>>>> fixed, wall

    >

    >     >>>>>>>>>> clock based intervals? This way you wouldn't

have to



give



    >

    >     >> up

    >

    >     >>>>>>>>>> the

    >

    >     >>>>>>>>>> notion

    >

    >     >>>>>>>>>> of stream time to be able to do periodic

processing.

    >

    >     >>>>>>>>>>

    >

    >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal

Borowiecki



wrote:



    >

    >     >>>>>>>>>>>

    >

    >     >>>>>>>>>>>

    >

    >     >>>>>>>>>>> Hi all,

    >

    >     >>>>>>>>>>>

    >

    >     >>>>>>>>>>> I have created a draft for KIP-138: Change



punctuate



    >

    >     >>>>>>>>>>> semantics

    >

    >     >>>>>>>>>>> <https://cwiki.apache.org/



confluence/display/KAFKA/KIP- <https://cwiki.apache.org/ confluence/display/KAFKA/KIP-><https://cwiki.apache.org/confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-><https://cwiki.apache.org/confluence/display/KAFKA/KIP->



    >

    >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI P-><https://cwiki.apache.org/confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-><https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>



138%



    >

    >     >>>>>>>>>>> 3A+C

    >

    >     >>>>>>>>>>> hange+

    >

    >     >>>>>>>>>>> punctuate+semantics>

    >

    >     >>>>>>>>>>> .

    >

    >     >>>>>>>>>>>

    >

    >     >>>>>>>>>>> Appreciating there can be different views

on



system-time



    >

    >     >> vs

    >

    >     >>>>>>>>>>> event-

    >

    >     >>>>>>>>>>> time

    >

    >     >>>>>>>>>>> semantics for punctuation depending on use-

case and



the



    >

    >     >>>>>>>>>>> importance of

    >

    >     >>>>>>>>>>> backwards compatibility of any such change,

I've



left it



    >

    >     >>>>>>>>>>> quite

    >

    >     >>>>>>>>>>> open

    >

    >     >>>>>>>>>>> and

    >

    >     >>>>>>>>>>> hope to fill in more info as the discussion



progresses.



    >

    >     >>>>>>>>>>>

    >

    >     >>>>>>>>>>> Thanks,

    >

    >     >>>>>>>>>>> Michal

    >

    >     >>>>>>> --

    >

    >     >>>>>>>

    >

    >     >>>>>>>

    >

    >     >>>>>>>     Tommy Becker

    >

    >     >>>>>>>

    >

    >     >>>>>>>     Senior Software Engineer

    >

    >     >>>>>>>

    >

    >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>

    >

    >     >>>>>>>

    >

    >     >>>>>>>     tivo.com

    >

    >     >>>>>>>

    >

    >     >>>>>>>

    >

    >     >>>>>>> ________________________________

    >

    >     >>>>>>>

    >

    >     >>>>>>> This email and any attachments may contain

confidential



and



    >

    >     >>>>>>> privileged material for the sole use of the

intended



recipient.



    >

    >     >> Any

    >

    >     >>>>>>> review, copying, or distribution of this email

(or any

    >

    >     >> attachments)

    >

    >     >>>>>>> by others is prohibited. If you are not the

intended



recipient,



    >

    >     >>>>>>> please contact the sender immediately and

permanently



delete this



    >

    >     >>>>>>> email and any attachments. No employee or agent

of TiVo



Inc. is



    >

    >     >>>>>>> authorized to conclude any binding agreement on

behalf



of TiVo



    >

    >     >> Inc.

    >

    >     >>>>>>> by email. Binding agreements with TiVo Inc. may

only be



made by a



    >

    >     >>>>>>> signed written agreement.

    >

    >     >>>>>>>

    >

    >     >>>>> --

    >

    >     >>>>>

    >

    >     >>>>>

    >

    >     >>>>>     Tommy Becker

    >

    >     >>>>>

    >

    >     >>>>>     Senior Software Engineer

    >

    >     >>>>>

    >

    >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>

    >

    >     >>>>>

    >

    >     >>>>>     tivo.com

    >

    >     >>>>>

    >

    >     >>>>>

    >

    >     >>>>> ________________________________

    >

    >     >>>>>

    >

    >     >>>>> This email and any attachments may contain

confidential



and



    >

    >     >> privileged

    >

    >     >>>>> material for the sole use of the intended

recipient. Any



review,



    >

    >     >>> copying,

    >

    >     >>>>> or distribution of this email (or any

attachments) by



others is



    >

    >     >>>> prohibited.

    >

    >     >>>>> If you are not the intended recipient, please

contact the



sender



    >

    >     >>>>> immediately and permanently delete this email and

any



attachments. No



    >

    >     >>>>> employee or agent of TiVo Inc. is authorized to

conclude



any binding



    >

    >     >>>>> agreement on behalf of TiVo Inc. by email.

Binding



agreements with



    >

    >     >> TiVo

    >

    >     >>>>> Inc. may only be made by a signed written

agreement.

    >

    >     >>>>>

    >

    >     >>>>

    >

    >     >>>

    >

    >     >>

    >

    >     >

    >

    >

    >

    >

    >

    >

    >

    >

    >

    > --

    >

    > <http://www.openbet.com/><http://www.openbet.com/> <http://www.openbet.com/><http://www.openbet.com/>



    >

    > *Michal Borowiecki*

    >

    > *Senior Software Engineer L4*

    >

    > *T: *

    >

    > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>

    >

    > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>

    >

    >

    >

    > *E: *

    >

    > michal.borowiecki@openbet.com<ma...@openbet.com>

    >

    > *W: *

    >

    > www.openbet.com<http://www.openbet.com>

    >

    > *OpenBet Ltd*

    >

    > Chiswick Park Building 9

    >

    > 566 Chiswick High Rd

    >

    > London

    >

    > W4 5XT

    >

    > UK

    >

    > <https://www.openbet.com/email_promo><https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo><https://www.openbet.com/email_promo>

    >

    >

    >

    > This message is confidential and intended only for the

addressee.



If you



    > have received this message in error, please immediately

notify the

    > postmaster@openbet.com<ma...@openbet.com> and delete it from your system as

well as



any



    > copies. The content of e-mails as well as traffic data may

be



monitored by



    > OpenBet for employment and security purposes. To protect

the



environment



    > please do not print this e-mail unless necessary. OpenBet

Ltd.



Registered



    > Office: Chiswick Park Building 9, 566 Chiswick High Road,

London,



W4 5XT,



    > United Kingdom. A company registered in England and Wales.



Registered no.



    > 3134634. VAT no. GB927523612

    >

    >

    >







--





    Tommy Becker



    Senior Software Engineer



    O +1 919.460.4747 <%28919%29%20460-4747>





    tivo.com







________________________________



This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.





--

<http://www.openbet.com/><http://www.openbet.com/> Michal Borowiecki

Senior Software Engineer L4

T: +44 208 742 1600 <+44%2020%208742%201600>







--

Signature

<http://www.openbet.com/><http://www.openbet.com/>     Michal Borowiecki

Senior Software Engineer L4

    T:     +44 208 742 1600





    +44 203 249 8448







   E:  michal.borowiecki@openbet.com<ma...@openbet.com>

   W:  www.openbet.com<http://www.openbet.com> <http://www.openbet.com/><http://www.openbet.com/>





    OpenBet Ltd



    Chiswick Park Building 9



    566 Chiswick High Rd



    London



    W4 5XT



    UK





<https://www.openbet.com/email_promo><https://www.openbet.com/email_promo>



This message is confidential and intended only for the addressee. If

you have received this message in error, please immediately notify the

postmaster@openbet.com<ma...@openbet.com> <ma...@openbet.com> and delete it

from your system as well as any copies. The content of e-mails as well

as traffic data may be monitored by OpenBet for employment and

security purposes. To protect the environment please do not print this

e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park

Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A

company registered in England and Wales. Registered no. 3134634. VAT

no. GB927523612



--

Signature

<http://www.openbet.com/><http://www.openbet.com/>   Michal Borowiecki

Senior Software Engineer L4

  T:      +44 208 742 1600





  +44 203 249 8448







  E:  michal.borowiecki@openbet.com<ma...@openbet.com>

  W:   www.openbet.com<http://www.openbet.com> <http://www.openbet.com/><http://www.openbet.com/>





  OpenBet Ltd



  Chiswick Park Building 9



  566 Chiswick High Rd



  London



  W4 5XT



  UK





<https://www.openbet.com/email_promo><https://www.openbet.com/email_promo>



This message is confidential and intended only for the addressee. If you

have received this message in error, please immediately notify the

postmaster@openbet.com<ma...@openbet.com> <ma...@openbet.com> and delete it

from your system as well as any copies. The content of e-mails as well

as traffic data may be monitored by OpenBet for employment and security

purposes. To protect the environment please do not print this e-mail

unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building

9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company

registered in England and Wales. Registered no. 3134634. VAT no.

GB927523612



--

Signature

<http://www.openbet.com/><http://www.openbet.com/>    Michal Borowiecki

Senior Software Engineer L4

       T:      +44 208 742 1600





       +44 203 249 8448







       E:        michal.borowiecki@openbet.com<ma...@openbet.com>

       W:      www.openbet.com<http://www.openbet.com> <http://www.openbet.com/><http://www.openbet.com/>





       OpenBet Ltd



       Chiswick Park Building 9



       566 Chiswick High Rd



       London



       W4 5XT



       UK





<https://www.openbet.com/email_promo><https://www.openbet.com/email_promo>



This message is confidential and intended only for the addressee. If you

have received this message in error, please immediately notify the

postmaster@openbet.com<ma...@openbet.com> <ma...@openbet.com> and delete it

from your system as well as any copies. The content of e-mails as well

as traffic data may be monitored by OpenBet for employment and security

purposes. To protect the environment please do not print this e-mail

unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building

9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company

registered in England and Wales. Registered no. 3134634. VAT no.

GB927523612



--

Signature

<http://www.openbet.com/><http://www.openbet.com/>  Michal Borowiecki

Senior Software Engineer L4

   T:      +44 208 742 1600





   +44 203 249 8448







   E:      michal.borowiecki@openbet.com<ma...@openbet.com>

   W:      www.openbet.com<http://www.openbet.com> <http://www.openbet.com/><http://www.openbet.com/>





   OpenBet Ltd



   Chiswick Park Building 9



   566 Chiswick High Rd



   London



   W4 5XT



   UK





<https://www.openbet.com/email_promo><https://www.openbet.com/email_promo>



This message is confidential and intended only for the addressee. If you

have received this message in error, please immediately notify the

postmaster@openbet.com<ma...@openbet.com> <ma...@openbet.com> and delete it

from your system as well as any copies. The content of e-mails as well

as traffic data may be monitored by OpenBet for employment and security

purposes. To protect the environment please do not print this e-mail

unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building

9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company

registered in England and Wales. Registered no. 3134634. VAT no.

GB927523612



--

[cid:image001.gif@01D2BED8.5D60E410]<http://www.openbet.com/>

Michal Borowiecki

Senior Software Engineer L4



T:

+44 208 742 1600


+44 203 249 8448






E:

michal.borowiecki@openbet.com<ma...@openbet.com>



W:

www.openbet.com<http://www.openbet.com/>




OpenBet Ltd


Chiswick Park Building 9


566 Chiswick High Rd


London


W4 5XT


UK


[cid:image003.gif@01D2BED8.5D60E410]<https://www.openbet.com/email_promo>


This message is confidential and intended only for the addressee. If you have received this message in error, please immediately notify the postmaster@openbet.com<ma...@openbet.com> and delete it from your system as well as any copies. The content of e-mails as well as traffic data may be monitored by OpenBet for employment and security purposes. To protect the environment please do not print this e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company registered in England and Wales. Registered no. 3134634. VAT no. GB927523612



Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Sorry, my original point was lost or I didn't make it clear enough.

I don't challenge the need for a system-time interval upper bound, one 
we don't have now. On the contrary my own use-case depends on it. 
However, the hybrid does provide a system-time interval upper bound, 
even in the absence of records, it only fails to provide a lower bound.

As to heartbeats, IMO that wouldn't need a lower bound on the heartbeat 
interval, an upper bound should suffice.


Nevertheless, I think I found a flaw in the hybrid proposal, as follows:

In the current stream-time punctuation scheme, the first 
PunctuationSchedule is set to 0 + interval (what else, since we haven't 
seen any records yet), but when triggered, it passes the current stream 
time as timestamp argument and uses that to schedule the next 
punctuation. It all makes sense.

In the hybrid this would only work if the first punctuation is triggered 
by stream time advance but could have unintended consequences if the 
first triggering is by system time.

Imagine this scenario: my apps are down and I have loads of records to 
process (perhaps I reset offsets to reprocess data).

If I start my streams app but records don't start coming in immediately 
(perhaps there is another app upstream that I have yet to start), then 
the first punctuation will be triggered by system time. It won't have 
any stream-time to use so, the only timestamp it can use is the 
system-time timestamp, both as an argument to punctuate and for the next 
schedule.

But if then messages start flowing and have event time way in the past 
(assuming event/ingestion time extractor), then stream time will be 
behind system-time until the stream catches up (which may be never if I 
just want to run in historical data as an experiment/validation). 
However, punctuate (having been triggered first by system time) will 
keep on recurring at regular intervals and always passing in the system 
time (or to be precise: the system time of previous punctuation + 
interval). That is surely not what is desired or expected, given the 
steady inflow of records.

Arun, have you implemented any special handling for this case in your 
system?

As mentioned before the hybrid punctuation scheme is not easy to reason 
about, I can't deny that.


Happy to stick with the 2 PunctuationTypes proposal (stream time and 
system time). I'll try to write up how users can implement hybrid-like 
semantics themselves on top of that to address those demanding use-cases 
mentioned on the wiki.

Maybe it won't end up too complex after all, but as per the mantra, it's 
enough that it's made possible ;-)

Thanks,

Michal


On 24/04/17 18:22, Matthias J. Sax wrote:
>>> Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?
> If you go with stream-time and don't have any input records for all
> partitions, punctuate would not be called at all, and thus your
> dashboard would "freeze".
>
>>> I thought about cron-type things, but aren't they better triggered by an
>>> external scheduler (they're more flexible anyway), which then feeds
>>> "commands" into the topology?
> I guess it depends what kind of periodic action you want to trigger. The
> "cron job" was just an analogy. Maybe it's just a heartbeat to some
> other service, that signals that your Streams app is still running.
>
>
> -Matthias
>
>
> On 4/24/17 10:02 AM, Michal Borowiecki wrote:
>> Thanks!
>>
>> Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?
>>
>> Unless too frequent messages on replay could overpower it?
>>
>>
>> I thought about cron-type things, but aren't they better triggered by an
>> external scheduler (they're more flexible anyway), which then feeds
>> "commands" into the topology?
>>
>> Just my 2c.
>>
>> Cheers,
>>
>> Michal
>>
>>
>> On 24/04/17 17:57, Matthias J. Sax wrote:
>>> A simple example would be some dashboard app, that needs to get
>>> "current" status in regular time intervals (ie, and real-time app).
>>>
>>> Or something like a "scheduler" -- think "cron job" application.
>>>
>>>
>>> -Matthias
>>>
>>> On 4/24/17 2:23 AM, Michal Borowiecki wrote:
>>>> Hi Matthias,
>>>>
>>>> I agree it's difficult to reason about the hybrid approach, I certainly
>>>> found it hard and I'm totally on board with the mantra.
>>>>
>>>> I'd be happy to limit the scope of this KIP to add system-time
>>>> punctuation semantics (in addition to existing stream-time semantics)
>>>> and leave more complex schemes for users to implement on top of that.
>>>>
>>>> Further additional PunctuationTypes, could then be added by future KIPs,
>>>> possibly including the hybrid approach once it has been given more thought.
>>>>
>>>>> There are real-time applications, that want to get
>>>>> callbacks in regular system-time intervals (completely independent from
>>>>> stream-time).
>>>> Can you please describe what they are, so that I can put them on the
>>>> wiki for later reference?
>>>>
>>>> Thanks,
>>>>
>>>> Michal
>>>>
>>>>
>>>> On 23/04/17 21:27, Matthias J. Sax wrote:
>>>>> Hi,
>>>>>
>>>>> I do like Damian's API proposal about the punctuation callback function.
>>>>>
>>>>> I also did reread the KIP and thought about the semantics we want to
>>>>> provide.
>>>>>
>>>>>> Given the above, I don't see a reason any more for a separate system-time based punctuation.
>>>>> I disagree here. There are real-time applications, that want to get
>>>>> callbacks in regular system-time intervals (completely independent from
>>>>> stream-time). Thus we should allow this -- if we really follow the
>>>>> "hybrid" approach, this could be configured with stream-time interval
>>>>> infinite and delay whatever system-time punctuation interval you want to
>>>>> have. However, I would like to add a proper API for this and do this
>>>>> configuration under the hood (that would allow one implementation within
>>>>> all kind of branching for different cases).
>>>>>
>>>>> Thus, we definitely should have PunctutionType#StreamTime and
>>>>> #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
>>>>> a fan of your latest API proposal.
>>>>>
>>>>>
>>>>> About the hybrid approach in general. On the one hand I like it, on the
>>>>> other hand, it seems to be rather (1) complicated (not necessarily from
>>>>> an implementation point of view, but for people to understand it) and
>>>>> (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
>>>>>
>>>>>> It may appear complicated at first but I do think these semantics will
>>>>>> still be more understandable to users than having 2 separate punctuation
>>>>>> schedules/callbacks with different PunctuationTypes.
>>>>> This statement only holds if you apply strong assumptions that I don't
>>>>> believe hold in general -- see (2) for details -- and I think it is
>>>>> harder than you assume to reason about the hybrid approach in general.
>>>>> IMHO, the hybrid approach is a "false friend" that seems to be easy to
>>>>> reason about...
>>>>>
>>>>>
>>>>> (1) Streams always embraced "easy to use" and we should really be
>>>>> careful to keep it this way. On the other hand, as we are talking about
>>>>> changes to PAPI, it won't affect DSL users (DSL does not use punctuation
>>>>> at all at the moment), and thus, the "easy to use" mantra might not be
>>>>> affected, while it will allow advanced users to express more complex stuff.
>>>>>
>>>>> I like the mantra: "make simple thing easy and complex things possible".
>>>>>
>>>>> (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
>>>>> implicit assumption that even-time progresses at the same "speed" as
>>>>> system-time during regular processing. This implies the assumption that
>>>>> a slower progress in stream-time indicates the absence of input events
>>>>> (and that later arriving input events will have a larger event-time with
>>>>> high probability). Even if this might be true for some use cases, I
>>>>> doubt it holds in general. Assume that you get a spike in traffic and
>>>>> for some reason stream-time does advance slowly because you have more
>>>>> records to process. This might trigger a system-time based punctuation
>>>>> call even if this seems not to be intended. I strongly believe that it
>>>>> is not easy to reason about the semantics of the hybrid approach (even
>>>>> if the intentional semantics would be super useful -- but I doubt that
>>>>> we get want we ask for).
>>>>>
>>>>> Thus, I also believe that one might need different "configuration"
>>>>> values for the hybrid approach if you run the same code for different
>>>>> scenarios: regular processing, re-processing, catching up scenario. And
>>>>> as the term "configuration" implies, we might be better off to not mix
>>>>> configuration with business logic that is expressed via code.
>>>>>
>>>>>
>>>>> One more comment: I also don't think that the hybrid approach is
>>>>> deterministic as claimed in the use-case subpage. I understand the
>>>>> reasoning and agree, that it is deterministic if certain assumptions
>>>>> hold -- compare above -- and if configured correctly. But strictly
>>>>> speaking it's not because there is a dependency on system-time (and
>>>>> IMHO, if system-time is involved it cannot be deterministic by definition).
>>>>>
>>>>>
>>>>>> I see how in theory this could be implemented on top of the 2 punctuate
>>>>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>>>>> the other system-time based) but it would be a much more complicated
>>>>>> scheme and I don't want to suggest that.
>>>>> I agree that expressing the intended hybrid semantics is harder if we
>>>>> offer only #StreamTime and #SystemTime punctuation. However, I also
>>>>> believe that the hybrid approach is a "false friend" with regard to
>>>>> reasoning about the semantics (it indicates that it more easy as it is
>>>>> in reality). Therefore, we might be better off to not offer the hybrid
>>>>> approach and make it clear to a developed, that it is hard to mix
>>>>> #StreamTime and #SystemTime in a semantically sound way.
>>>>>
>>>>>
>>>>> Looking forward to your feedback. :)
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 4/22/17 11:43 AM, Michal Borowiecki wrote:
>>>>>> Hi all,
>>>>>>
>>>>>> Looking for feedback on the functional interface approach Damian
>>>>>> proposed. What do people think?
>>>>>>
>>>>>> Further on the semantics of triggering punctuate though:
>>>>>>
>>>>>> I ran through the 2 use cases that Arun had kindly put on the wiki
>>>>>> (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
>>>>>> in my head and on a whiteboard and I can't find a better solution than
>>>>>> the "hybrid" approach he had proposed.
>>>>>>
>>>>>> I see how in theory this could be implemented on top of the 2 punctuate
>>>>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>>>>> the other system-time based) but it would be a much more complicated
>>>>>> scheme and I don't want to suggest that.
>>>>>>
>>>>>> However, to add to the hybrid algorithm proposed, I suggest one
>>>>>> parameter to that: a tolerance period, expressed in milliseconds
>>>>>> system-time, after which the punctuation will be invoked in case the
>>>>>> stream-time advance hasn't triggered it within the requested interval
>>>>>> since the last invocation of punctuate (as recorded in system-time)
>>>>>>
>>>>>> This would allow a user-defined tolerance for late arriving events. The
>>>>>> trade off would be left for the user to decide: regular punctuation in
>>>>>> the case of absence of events vs allowing for records arriving late or
>>>>>> some build-up due to processing not catching up with the event rate.
>>>>>> In the one extreme, this tolerance could be set to infinity, turning
>>>>>> hybrid into simply stream-time based punctuate, like we have now. In the
>>>>>> other extreme, the tolerance could be set to 0, resulting in a
>>>>>> system-time upper bound on the effective punctuation interval.
>>>>>>
>>>>>> Given the above, I don't see a reason any more for a separate
>>>>>> system-time based punctuation. The "hybrid" approach with 0ms tolerance
>>>>>> would under normal operation trigger at regular intervals wrt the
>>>>>> system-time, except in cases of re-play/catch-up, where the stream time
>>>>>> advances faster than system time. In these cases punctuate would happen
>>>>>> more often than the specified interval wrt system time. However, the
>>>>>> use-cases that need system-time punctuations (that I've seen at least)
>>>>>> really only have a need for an upper bound on punctuation delay but
>>>>>> don't need a lower bound.
>>>>>>
>>>>>> To that effect I'd propose the api to be as follows, on ProcessorContext:
>>>>>>
>>>>>> schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
>>>>>>
>>>>>> schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
>>>>>>
>>>>>> Punctuation is triggered when either:
>>>>>> - the stream time advances past the (stream time of the previous
>>>>>> punctuation) + interval;
>>>>>> - or (iff the toleranceInterval is set) when the system time advances
>>>>>> past the (system time of the previous punctuation) + interval +
>>>>>> toleranceInterval
>>>>>>
>>>>>> In either case:
>>>>>> - we trigger punctuate passing as the argument the stream time at which
>>>>>> the current punctuation was meant to happen
>>>>>> - next punctuate is scheduled at (stream time at which the current
>>>>>> punctuation was meant to happen) + interval
>>>>>>
>>>>>> It may appear complicated at first but I do think these semantics will
>>>>>> still be more understandable to users than having 2 separate punctuation
>>>>>> schedules/callbacks with different PunctuationTypes.
>>>>>>
>>>>>>
>>>>>>
>>>>>> PS. Having re-read this, maybe the following alternative would be easier
>>>>>> to understand (WDYT?):
>>>>>>
>>>>>> schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
>>>>>>
>>>>>> schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
>>>>>>
>>>>>> Punctuation is triggered when either:
>>>>>> - the stream time advances past the (stream time of the previous
>>>>>> punctuation) + streamTimeInterval;
>>>>>> - or (iff systemTimeUpperBound is set) when the system time advances
>>>>>> past the (system time of the previous punctuation) + systemTimeUpperBound
>>>>>>
>>>>>> Awaiting comments.
>>>>>>
>>>>>> Thanks,
>>>>>> Michal
>>>>>>
>>>>>> On 21/04/17 16:56, Michal Borowiecki wrote:
>>>>>>> Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>>>>>>> in favour of something that doesn't return a record. Not a problem though.
>>>>>>>
>>>>>>>
>>>>>>> On 21/04/17 16:32, Damian Guy wrote:
>>>>>>>> Thanks Michal,
>>>>>>>> I agree Transformer.punctuate should also be void, but we can deprecate
>>>>>>>> that too in favor of the new interface.
>>>>>>>>
>>>>>>>> Thanks for the javadoc PR!
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Damian
>>>>>>>>
>>>>>>>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>>>
>>>>>>>>> Yes, that looks better to me.
>>>>>>>>>
>>>>>>>>> Note that punctuate on Transformer is currently returning a record, but I
>>>>>>>>> think it's ok to have all output records be sent via
>>>>>>>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>>>>>>>> multiple records from one invocation of punctuate.
>>>>>>>>>
>>>>>>>>> This way it's consistent between Processor and Transformer.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>>>>>>>> there:
>>>>>>>>>
>>>>>>>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>>>>>>>
>>>>>>>>> and PR: https://github.com/apache/kafka/pull/2884
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>>
>>>>>>>>> Michal
>>>>>>>>> On 20/04/17 18:55, Damian Guy wrote:
>>>>>>>>>
>>>>>>>>> Hi Michal,
>>>>>>>>>
>>>>>>>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>>>>>>>> the API.
>>>>>>>>> 1. deprecate the punctuate method on Processor
>>>>>>>>> 2. create a new Functional Interface just for Punctuation, something like:
>>>>>>>>> interface Punctuator {
>>>>>>>>>      void punctuate(long timestamp)
>>>>>>>>> }
>>>>>>>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>>>>>>>> interval, PunctuationType type, Punctuator callback)
>>>>>>>>> 4. deprecate the existing schedule function
>>>>>>>>>
>>>>>>>>> Thoughts?
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>>>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Thomas,
>>>>>>>>>>
>>>>>>>>>> I would say our use cases fall in the same category as yours.
>>>>>>>>>>
>>>>>>>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>>>>>>>
>>>>>>>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>>>>>>>> of design:
>>>>>>>>>>
>>>>>>>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>>>>>>>
>>>>>>>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>>>>>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>>>>>>>
>>>>>>>>>> Since the output field X is derived in some non-trivial way, we don't
>>>>>>>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>>>>>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>>>>>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>>>>>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>>>>>>>> another input field Z).
>>>>>>>>>>
>>>>>>>>>> So we have kv stores with the records and an additional kv store with
>>>>>>>>>> timestamp->id mapping which act like an index where we periodically do a
>>>>>>>>>> ranged query.
>>>>>>>>>>
>>>>>>>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>>>>>>>> work when there were no regular msgs on the input topic.
>>>>>>>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>>>>>>>> outside we created a "ticker" that produced msgs once per second onto
>>>>>>>>>> another topic and fed it into the same topology to trigger punctuate.
>>>>>>>>>> This didn't work either, which was much more surprising to us at the
>>>>>>>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>>>>>>>> *all* input partitions receive messages regularly.
>>>>>>>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>>>>>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>>>>>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>>>>>>>> is consumed by the second topology and is its only input topic. There's a
>>>>>>>>>> transformer on that topic which populates and updates the time-based
>>>>>>>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>>>>>>>> elapsed, the record id is sent to the main transformer, which
>>>>>>>>>> updates/deletes the record from the main kv store and forwards the
>>>>>>>>>> transformed record to the output topic.
>>>>>>>>>>
>>>>>>>>>> To me this setup feels horrendously complicated for what it does.
>>>>>>>>>>
>>>>>>>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>>>>>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>>>>>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>>>>>>>> However, the ticks don't feel so hacky when you realise they give you
>>>>>>>>>> some hypothetical benefits in predictability. You can reprocess the
>>>>>>>>>> messages in a reproducible manner, since the topologies use event-time,
>>>>>>>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>>>>>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>>>>>>>> reprocessing).
>>>>>>>>>> To make that work though, we would have to have the stream time advance
>>>>>>>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>>>>>>>> presence of messages on the other input topic.
>>>>>>>>>>
>>>>>>>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>>>>>>>> and the hybrid would work to simplify this a lot.
>>>>>>>>>>
>>>>>>>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>>>>>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>>>>>>>> have to go with punctuate there too.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Michal
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>>>>>>>
>>>>>>>>>> Here's an example that we currently have.  We have a streams processor
>>>>>>>>>> that does a transform from one topic into another. One of the fields in
>>>>>>>>>> the source topic record is an expiration time, and one of the functions
>>>>>>>>>> of the processor is to ensure that expired records get deleted promptly
>>>>>>>>>> after that time passes (typically days or weeks after the message was
>>>>>>>>>> originally produced). To do that, the processor keeps a state store of
>>>>>>>>>> keys and expiration times, iterates that store in punctuate(), and
>>>>>>>>>> emits delete (null) records for expired items. This needs to happen at
>>>>>>>>>> some minimum interval regardless of the incoming message rate of the
>>>>>>>>>> source topic.
>>>>>>>>>>
>>>>>>>>>> In this scenario, the expiration of records is the primary function of
>>>>>>>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>>>>>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>>>>>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>>>>>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>>>>>>>> range of use-cases.
>>>>>>>>>>
>>>>>>>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>>>>>>>
>>>>>>>>>> I apologize for the longer email below.  To my defense, it started
>>>>>>>>>> out much
>>>>>>>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>>>>>>>> devil's
>>>>>>>>>> advocate for a number of arguments brought forth in order to help
>>>>>>>>>> improve
>>>>>>>>>> this KIP -- I am not implying I necessarily disagree with the
>>>>>>>>>> arguments.
>>>>>>>>>>
>>>>>>>>>> That aside, here are some further thoughts.
>>>>>>>>>>
>>>>>>>>>> First, there are (at least?) two categories for actions/behavior you
>>>>>>>>>> invoke
>>>>>>>>>> via punctuate():
>>>>>>>>>>
>>>>>>>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>>>>>>>> to
>>>>>>>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>>>>>>>> the
>>>>>>>>>> impact of punctuate is typically not observable by other processing
>>>>>>>>>> nodes
>>>>>>>>>> in the topology.
>>>>>>>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>>>>>>>> the
>>>>>>>>>> punctuate is all about being observable by downstream processing
>>>>>>>>>> nodes.
>>>>>>>>>>
>>>>>>>>>> A few releases back, we introduced record caches (DSL) and state
>>>>>>>>>> store
>>>>>>>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>>>>>>>> relating to
>>>>>>>>>> (2) where some users needed to control -- here: limit -- the
>>>>>>>>>> downstream
>>>>>>>>>> output rate of Kafka Streams because the downstream systems/apps
>>>>>>>>>> would not
>>>>>>>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>>>>>>>> their
>>>>>>>>>> scalability).  The argument for KIP-63, which notably did not
>>>>>>>>>> introduce a
>>>>>>>>>> "trigger" API, was that such an interaction with downstream systems
>>>>>>>>>> is an
>>>>>>>>>> operational concern;  it should not impact the processing *logic* of
>>>>>>>>>> your
>>>>>>>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>>>>>>>> API,
>>>>>>>>>> especially not the declarative DSL, with such operational concerns.
>>>>>>>>>>
>>>>>>>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>>>>>>>> sorry, I
>>>>>>>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>>>>>>>> observing that our conversation is moving more and more into the
>>>>>>>>>> direction
>>>>>>>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>>>>>>>> for
>>>>>>>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>>>>>>>> comments voiced here are about sth like "IF stream-time didn't
>>>>>>>>>> trigger
>>>>>>>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>>>>>>>> want
>>>>>>>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>>>>>>>> related
>>>>>>>>>> note, whatever we are discussing here will impact state store caches
>>>>>>>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>>>>>>>> should
>>>>>>>>>> clarify any such impact here.
>>>>>>>>>>
>>>>>>>>>> Switching topics slightly.
>>>>>>>>>>
>>>>>>>>>> Jay wrote:
>>>>>>>>>>
>>>>>>>>>> One thing I've always found super important for this kind of design
>>>>>>>>>> work
>>>>>>>>>> is to do a really good job of cataloging the landscape of use cases
>>>>>>>>>> and
>>>>>>>>>> how prevalent each one is.
>>>>>>>>>>
>>>>>>>>>> +1 to this, as others have already said.
>>>>>>>>>>
>>>>>>>>>> Here, let me highlight -- just in case -- that when we talked about
>>>>>>>>>> windowing use cases in the recent emails, the Processor API (where
>>>>>>>>>> `punctuate` resides) does not have any notion of windowing at
>>>>>>>>>> all.  If you
>>>>>>>>>> want to do windowing *in the Processor API*, you must do so manually
>>>>>>>>>> in
>>>>>>>>>> combination with window stores.  For this reason I'd suggest to
>>>>>>>>>> discuss use
>>>>>>>>>> cases not just in general, but also in view of how you'd do so in the
>>>>>>>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>>>>>>>> `punctuate`
>>>>>>>>>> does not impact the DSL at all, unless we add new functionality to
>>>>>>>>>> it.
>>>>>>>>>>
>>>>>>>>>> Jay wrote in his strawman example:
>>>>>>>>>>
>>>>>>>>>> You aggregate click and impression data for a reddit like site.
>>>>>>>>>> Every ten
>>>>>>>>>> minutes you want to output a ranked list of the top 10 articles
>>>>>>>>>> ranked by
>>>>>>>>>> clicks/impressions for each geographical area. I want to be able
>>>>>>>>>> run this
>>>>>>>>>> in steady state as well as rerun to regenerate results (or catch up
>>>>>>>>>> if it
>>>>>>>>>> crashes).
>>>>>>>>>>
>>>>>>>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>>>>>>>> we
>>>>>>>>>> argued that the reason for saying "every ten minutes" above is not
>>>>>>>>>> necessarily about because you want to output data *exactly* after ten
>>>>>>>>>> minutes, but that you want to perform an aggregation based on 10-
>>>>>>>>>> minute
>>>>>>>>>> windows of input data; i.e., the point is about specifying the input
>>>>>>>>>> for
>>>>>>>>>> your aggregation, not or less about when the results of the
>>>>>>>>>> aggregation
>>>>>>>>>> should be send downstream.  To take an extreme example, you could
>>>>>>>>>> disable
>>>>>>>>>> record caches and let your app output a downstream update for every
>>>>>>>>>> incoming input record.  If the last input record was from at minute 7
>>>>>>>>>> of 10
>>>>>>>>>> (for a 10-min window), then what your app would output at minute 10
>>>>>>>>>> would
>>>>>>>>>> be identical to what it had already emitted at minute 7 earlier
>>>>>>>>>> anyways.
>>>>>>>>>> This is particularly true when we take late-arriving data into
>>>>>>>>>> account:  if
>>>>>>>>>> a late record arrived at minute 13, your app would (by default) send
>>>>>>>>>> a new
>>>>>>>>>> update downstream, even though the "original" 10 minutes have already
>>>>>>>>>> passed.
>>>>>>>>>>
>>>>>>>>>> Jay wrote...:
>>>>>>>>>>
>>>>>>>>>> There are a couple of tricky things that seem to make this hard
>>>>>>>>>> with
>>>>>>>>>>
>>>>>>>>>> either
>>>>>>>>>>
>>>>>>>>>> of the options proposed:
>>>>>>>>>> 1. If I emit this data using event time I have the problem
>>>>>>>>>> described where
>>>>>>>>>> a geographical region with no new clicks or impressions will fail
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>> output
>>>>>>>>>>
>>>>>>>>>> results.
>>>>>>>>>>
>>>>>>>>>> ...and Arun Mathew wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> We window by the event time, but trigger punctuate in <punctuate
>>>>>>>>>> interval>
>>>>>>>>>> duration of system time, in the absence of an event crossing the
>>>>>>>>>> punctuate
>>>>>>>>>> event time.
>>>>>>>>>>
>>>>>>>>>> So, given what I wrote above about the status quo and what you can
>>>>>>>>>> already
>>>>>>>>>> do with it, is the concern that the state store cache doesn't give
>>>>>>>>>> you
>>>>>>>>>> *direct* control over "forcing an output after no later than X
>>>>>>>>>> seconds [of
>>>>>>>>>> processing-time]" but only indirect control through a cache
>>>>>>>>>> size?  (Note
>>>>>>>>>> that I am not dismissing the claims why this might be helpful.)
>>>>>>>>>>
>>>>>>>>>> Arun Mathew wrote:
>>>>>>>>>>
>>>>>>>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>>>>>>>> output the
>>>>>>>>>> event counts on each topic on each cluster aggregated over a 1
>>>>>>>>>> minute
>>>>>>>>>> window. We have to use event time to be able to cross check the
>>>>>>>>>> counts.
>>>>>>>>>>
>>>>>>>>>> But
>>>>>>>>>>
>>>>>>>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>>>>>>>> time in
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> absence of events. Otherwise the event counts for unexpired windows
>>>>>>>>>> would
>>>>>>>>>> be 0 which is bad.
>>>>>>>>>>
>>>>>>>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>>>>>>>> absence
>>>>>>>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>>>>>>>> Scala
>>>>>>>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>>>>>>>> root
>>>>>>>>>> cause that the downstream system interprets the absence of output in
>>>>>>>>>> a
>>>>>>>>>> particular way ("No output after 1 minute = I consider the output to
>>>>>>>>>> be
>>>>>>>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>>>>>>>> possible)
>>>>>>>>>> to correctly handle the difference between absence of output vs.
>>>>>>>>>> output of
>>>>>>>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>>>>>>>> but
>>>>>>>>>> want to understand the motivation better. :-)
>>>>>>>>>>
>>>>>>>>>> Also, to add some perspective, in some related discussions we talked
>>>>>>>>>> about
>>>>>>>>>> how a Kafka Streams application should not worry or not be coupled
>>>>>>>>>> unnecessarily with such interpretation specifics in a downstream
>>>>>>>>>> system's
>>>>>>>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>>>>>>>> more
>>>>>>>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>>>>>>>> than
>>>>>>>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>>>>>>>> and
>>>>>>>>>> downstream systems, including helping to reconcile the differences in
>>>>>>>>>> how
>>>>>>>>>> these systems interpret changes, updates, late-arriving data,
>>>>>>>>>> etc.  Kafka
>>>>>>>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>>>>>>>> processing from the specifics of downstream systems, thanks to
>>>>>>>>>> specific
>>>>>>>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>>>>>>>> this
>>>>>>>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>>>>>>>> but
>>>>>>>>>> it's currently awkward to use Connect for this", this might be a
>>>>>>>>>> problem we
>>>>>>>>>> can solve, too.)
>>>>>>>>>>
>>>>>>>>>> Switching topics slightly again.
>>>>>>>>>>
>>>>>>>>>> Thomas wrote:
>>>>>>>>>>
>>>>>>>>>> I'm not entirely convinced that a separate callback (option C)
>>>>>>>>>> is that messy (it could just be a default method with an empty
>>>>>>>>>> implementation), but if we wanted a single API to handle both
>>>>>>>>>> cases,
>>>>>>>>>> how about something like the following?
>>>>>>>>>>
>>>>>>>>>> enum Time {
>>>>>>>>>>     STREAM,
>>>>>>>>>>     CLOCK
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>>>>>>>> then
>>>>>>>>>> whatever the user is doing inside this method is a black box to Kafka
>>>>>>>>>> Streams (similar to how we have no idea what the user does inside a
>>>>>>>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>>>>>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>>>>>>>> won't
>>>>>>>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>>>>>>>> frequency than the processing-time action.  Or, we won't know whether
>>>>>>>>>> the
>>>>>>>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>>>>>>>> minute of
>>>>>>>>>> stream-time or 1-minute of processing-time, whichever comes
>>>>>>>>>> first").  That
>>>>>>>>>> said, I am not certain yet whether we would need such knowledge
>>>>>>>>>> because,
>>>>>>>>>> when using the Processor API, most of the work and decisions must be
>>>>>>>>>> done
>>>>>>>>>> by the user anyways.  It would matter though if the concept of
>>>>>>>>>> "triggers"
>>>>>>>>>> were to bubble up into the DSL because in the DSL the management of
>>>>>>>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>>>>>>>> Streams.
>>>>>>>>>>
>>>>>>>>>> [In any case, btw, we have the corner case where the user configured
>>>>>>>>>> the
>>>>>>>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>>>>>>>> extractor), at which point both punctuate variants are based on the
>>>>>>>>>> same
>>>>>>>>>> time semantics / timeline.]
>>>>>>>>>>
>>>>>>>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>>>>>>>> it
>>>>>>>>>> this far. :-)
>>>>>>>>>>
>>>>>>>>>> More than happy to hear your thoughts!
>>>>>>>>>> Michael
>>>>>>>>>>
>>>>>>>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks Matthias.
>>>>>>>>>> Sure, will correct it right away.
>>>>>>>>>>
>>>>>>>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> Thanks for preparing this page!
>>>>>>>>>>
>>>>>>>>>> About terminology:
>>>>>>>>>>
>>>>>>>>>> You introduce the term "event time" -- but we should call this
>>>>>>>>>> "stream
>>>>>>>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>>>>>>>> this
>>>>>>>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>>>>>>>
>>>>>>>>>> Does this make sense to you?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>>>>>>
>>>>>>>>>> Thanks Ewen.
>>>>>>>>>>
>>>>>>>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>>>>>>>
>>>>>>>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>>>>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>>>>>>>> public.
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> Arun Mathew
>>>>>>>>>>
>>>>>>>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>      Arun,
>>>>>>>>>>
>>>>>>>>>>      I've given you permission to edit the wiki. Let me know if
>>>>>>>>>> you run
>>>>>>>>>>
>>>>>>>>>> into any
>>>>>>>>>>
>>>>>>>>>>      issues.
>>>>>>>>>>
>>>>>>>>>>      -Ewen
>>>>>>>>>>
>>>>>>>>>>      On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp> <am...@yahoo-corp.jp>
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>      > Thanks Michal. I don\u2019t have the access yet [arunmathew88].
>>>>>>>>>> Should I
>>>>>>>>>>
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>>      > sending a separate mail for this?
>>>>>>>>>>      >
>>>>>>>>>>      > I thought one of the person following this thread would be
>>>>>>>>>> able to
>>>>>>>>>>
>>>>>>>>>> give me
>>>>>>>>>>
>>>>>>>>>>      > access.
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>>>>>>>>>      > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>>>>      > *Date: *Friday, April 7, 2017 at 17:16
>>>>>>>>>>      > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>>>>      > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>>>>>>> semantics
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > Hi Arun,
>>>>>>>>>>      >
>>>>>>>>>>      > I was thinking along the same lines as you, listing the use
>>>>>>>>>> cases
>>>>>>>>>>
>>>>>>>>>> on the
>>>>>>>>>>
>>>>>>>>>>      > wiki, but didn't find time to get around doing that yet.
>>>>>>>>>>      > Don't mind if you do it if you have access now.
>>>>>>>>>>      > I was thinking it would be nice if, once we have the use
>>>>>>>>>> cases
>>>>>>>>>>
>>>>>>>>>> listed,
>>>>>>>>>>
>>>>>>>>>>      > people could use likes to up-vote the use cases similar to
>>>>>>>>>> what
>>>>>>>>>>
>>>>>>>>>> they're
>>>>>>>>>>
>>>>>>>>>>      > working on.
>>>>>>>>>>      >
>>>>>>>>>>      > I should have a bit more time to action this in the next
>>>>>>>>>> few days,
>>>>>>>>>>
>>>>>>>>>> but
>>>>>>>>>>
>>>>>>>>>>      > happy for you to do it if you can beat me to it ;-)
>>>>>>>>>>      >
>>>>>>>>>>      > Cheers,
>>>>>>>>>>      > Michal
>>>>>>>>>>      >
>>>>>>>>>>      > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>>>>>>      >
>>>>>>>>>>      > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > Of course. I was thinking of a subpage where people can
>>>>>>>>>>
>>>>>>>>>> collaborate.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > Will do as per Michael\u2019s suggestion.
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > Regards,
>>>>>>>>>>      >
>>>>>>>>>>      > Arun Mathew
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>>>>> <
>>>>>>>>>>
>>>>>>>>>> matthias@confluent.io> wrote:
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >     Please share your Wiki-ID and a committer can give you
>>>>>>>>>> write
>>>>>>>>>>
>>>>>>>>>> access.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >     Btw: as you did not initiate the KIP, you should not
>>>>>>>>>> change the
>>>>>>>>>>
>>>>>>>>>> KIP
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     without the permission of the original author -- in
>>>>>>>>>> this case
>>>>>>>>>>
>>>>>>>>>> Michael.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >     So you might also just share your thought over the
>>>>>>>>>> mailing list
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     Michael can update the KIP page. Or, as an alternative,
>>>>>>>>>> just
>>>>>>>>>>
>>>>>>>>>> create a
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     subpage for the KIP page.
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >     @Michael: WDYT?
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >     -Matthias
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>>>>>>      >
>>>>>>>>>>      >     > Hi Jay,
>>>>>>>>>>      >
>>>>>>>>>>      >     >           Thanks for the advise, I would like to list
>>>>>>>>>> down
>>>>>>>>>>
>>>>>>>>>> the use cases as
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > per your suggestion. But it seems I don't have write
>>>>>>>>>>
>>>>>>>>>> permission to the
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > Apache Kafka Confluent Space. Whom shall I request
>>>>>>>>>> for it?
>>>>>>>>>>      >
>>>>>>>>>>      >     >
>>>>>>>>>>      >
>>>>>>>>>>      >     > Regarding your last question. We are using a patch in
>>>>>>>>>> our
>>>>>>>>>>
>>>>>>>>>> production system
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > which does exactly this.
>>>>>>>>>>      >
>>>>>>>>>>      >     > We window by the event time, but trigger punctuate in
>>>>>>>>>>
>>>>>>>>>> <punctuate interval>
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > duration of system time, in the absence of an event
>>>>>>>>>> crossing
>>>>>>>>>>
>>>>>>>>>> the punctuate
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > event time.
>>>>>>>>>>      >
>>>>>>>>>>      >     >
>>>>>>>>>>      >
>>>>>>>>>>      >     > We are using Kafka Stream for our Audit Trail, where
>>>>>>>>>> we need
>>>>>>>>>>
>>>>>>>>>> to output the
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > event counts on each topic on each cluster aggregated
>>>>>>>>>> over a
>>>>>>>>>>
>>>>>>>>>> 1 minute
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > window. We have to use event time to be able to cross
>>>>>>>>>> check
>>>>>>>>>>
>>>>>>>>>> the counts. But
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > we need to trigger punctuate [aggregate event pushes]
>>>>>>>>>> by
>>>>>>>>>>
>>>>>>>>>> system time in the
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > absence of events. Otherwise the event counts for
>>>>>>>>>> unexpired
>>>>>>>>>>
>>>>>>>>>> windows would
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > be 0 which is bad.
>>>>>>>>>>      >
>>>>>>>>>>      >     >
>>>>>>>>>>      >
>>>>>>>>>>      >     > "Maybe a hybrid solution works: I window by event
>>>>>>>>>> time but
>>>>>>>>>>
>>>>>>>>>> trigger results
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > by system time for windows that have updated? Not
>>>>>>>>>> really sure
>>>>>>>>>>
>>>>>>>>>> the details
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > of making that work. Does that work? Are there
>>>>>>>>>> concrete
>>>>>>>>>>
>>>>>>>>>> examples where you
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > actually want the current behavior?"
>>>>>>>>>>      >
>>>>>>>>>>      >     >
>>>>>>>>>>      >
>>>>>>>>>>      >     > --
>>>>>>>>>>      >
>>>>>>>>>>      >     > With Regards,
>>>>>>>>>>      >
>>>>>>>>>>      >     >
>>>>>>>>>>      >
>>>>>>>>>>      >     > Arun Mathew
>>>>>>>>>>      >
>>>>>>>>>>      >     > Yahoo! JAPAN Corporation
>>>>>>>>>>      >
>>>>>>>>>>      >     >
>>>>>>>>>>      >
>>>>>>>>>>      >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>>>>>>>
>>>>>>>>>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >
>>>>>>>>>>      >
>>>>>>>>>>      >     >> Hi Jay,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> The hybrid solution is exactly what I expect and
>>>>>>>>>> need for
>>>>>>>>>>
>>>>>>>>>> our use cases
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> when dealing with telecom data.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> Thanks
>>>>>>>>>>      >
>>>>>>>>>>      >     >> Tianji
>>>>>>>>>>      >
>>>>>>>>>>      >     >>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>>>>>>>
>>>>>>>>>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> Hey guys,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> One thing I've always found super important for
>>>>>>>>>> this kind
>>>>>>>>>>
>>>>>>>>>> of design work
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> is
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> to do a really good job of cataloging the landscape
>>>>>>>>>> of use
>>>>>>>>>>
>>>>>>>>>> cases and how
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> prevalent each one is. By that I mean not just
>>>>>>>>>> listing lots
>>>>>>>>>>
>>>>>>>>>> of uses, but
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> also grouping them into categories that
>>>>>>>>>> functionally need
>>>>>>>>>>
>>>>>>>>>> the same thing.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> In the absence of this it is very hard to reason
>>>>>>>>>> about
>>>>>>>>>>
>>>>>>>>>> design proposals.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> From the proposals so far I think we have a lot of
>>>>>>>>>>
>>>>>>>>>> discussion around
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> possible apis, but less around what the user needs
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> different use
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> cases
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> and how they would implement that using the api.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> Here is an example:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> You aggregate click and impression data for a
>>>>>>>>>> reddit like
>>>>>>>>>>
>>>>>>>>>> site. Every ten
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> minutes you want to output a ranked list of the top
>>>>>>>>>> 10
>>>>>>>>>>
>>>>>>>>>> articles ranked by
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> clicks/impressions for each geographical area. I
>>>>>>>>>> want to be
>>>>>>>>>>
>>>>>>>>>> able run this
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> in steady state as well as rerun to regenerate
>>>>>>>>>> results (or
>>>>>>>>>>
>>>>>>>>>> catch up if it
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> crashes).
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> There are a couple of tricky things that seem to
>>>>>>>>>> make this
>>>>>>>>>>
>>>>>>>>>> hard with
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> either
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> of the options proposed:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> 1. If I emit this data using event time I have the
>>>>>>>>>> problem
>>>>>>>>>>
>>>>>>>>>> described
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> where
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> a geographical region with no new clicks or
>>>>>>>>>> impressions
>>>>>>>>>>
>>>>>>>>>> will fail to
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> output
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> results.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> 2. If I emit this data using system time I have the
>>>>>>>>>> problem
>>>>>>>>>>
>>>>>>>>>> that when
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> reprocessing data my window may not be ten minutes
>>>>>>>>>> but 10
>>>>>>>>>>
>>>>>>>>>> hours if my
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> processing is very fast so it dramatically changes
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> output.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> Maybe a hybrid solution works: I window by event
>>>>>>>>>> time but
>>>>>>>>>>
>>>>>>>>>> trigger results
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> by system time for windows that have updated? Not
>>>>>>>>>> really
>>>>>>>>>>
>>>>>>>>>> sure the details
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> of making that work. Does that work? Are there
>>>>>>>>>> concrete
>>>>>>>>>>
>>>>>>>>>> examples where
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> you
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> actually want the current behavior?
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> -Jay
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>>>>>>>
>>>>>>>>>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> wrote:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> Hi All,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> Thanks for the KIP. We were also in need of a
>>>>>>>>>> mechanism to
>>>>>>>>>>
>>>>>>>>>> trigger
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> punctuate in the absence of events.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> As I described in [
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> plugin.system.issuetabpanels:comment-
>>>>>>>>>> tabpanel#comment-
>>>>>>>>>>
>>>>>>>>>> 15926036
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> ],
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - Our approached involved using the event time
>>>>>>>>>> by
>>>>>>>>>>
>>>>>>>>>> default.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - The method to check if there is any punctuate
>>>>>>>>>> ready
>>>>>>>>>>
>>>>>>>>>> in the
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    PunctuationQueue is triggered via the any event
>>>>>>>>>>
>>>>>>>>>> received by the
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> stream
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    tread, or at the polling intervals in the
>>>>>>>>>> absence of
>>>>>>>>>>
>>>>>>>>>> any events.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - When we create Punctuate objects (which
>>>>>>>>>> contains the
>>>>>>>>>>
>>>>>>>>>> next event
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> time
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    for punctuation and interval), we also record
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> creation time
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> (system
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    time).
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - While checking for maturity of Punctuate
>>>>>>>>>> Schedule by
>>>>>>>>>>      >
>>>>>>>>>>      >     >> mayBePunctuate
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    method, we also check if the system clock has
>>>>>>>>>> elapsed
>>>>>>>>>>
>>>>>>>>>> the punctuate
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    interval since the schedule creation time.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - In the absence of any event, or in the
>>>>>>>>>> absence of any
>>>>>>>>>>
>>>>>>>>>> event for
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> one
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    topic in the partition group assigned to the
>>>>>>>>>> stream
>>>>>>>>>>
>>>>>>>>>> task, the system
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> time
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    will elapse the interval and we trigger a
>>>>>>>>>> punctuate
>>>>>>>>>>
>>>>>>>>>> using the
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> expected
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    punctuation event time.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - we then create the next punctuation schedule
>>>>>>>>>> as
>>>>>>>>>>
>>>>>>>>>> punctuation event
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> time
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    + punctuation interval, [again recording the
>>>>>>>>>> system
>>>>>>>>>>
>>>>>>>>>> time of creation
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> of
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> the
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    schedule].
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>>>>>>> approach
>>>>>>>>>>
>>>>>>>>>> has pros and
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> cons.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> Pros
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - Punctuates will happen in <punctuate
>>>>>>>>>> interval> time
>>>>>>>>>>
>>>>>>>>>> duration at
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> max
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> in
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    terms of system time.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - The semantics as a whole continues to revolve
>>>>>>>>>> around
>>>>>>>>>>
>>>>>>>>>> event time.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - We can use the old data [old timestamps] to
>>>>>>>>>> rerun any
>>>>>>>>>>
>>>>>>>>>> experiments
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> or
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    tests.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> Cons
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - In case the  <punctuate interval> is not a
>>>>>>>>>> time
>>>>>>>>>>
>>>>>>>>>> duration [say
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> logical
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    time/event count], then the approach might not
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> meaningful.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - In case there is a case where we have to wait
>>>>>>>>>> for an
>>>>>>>>>>
>>>>>>>>>> actual event
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> from
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    a low event rate partition in the partition
>>>>>>>>>> group, this
>>>>>>>>>>
>>>>>>>>>> approach
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> will
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> jump
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    the gun.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    - in case the event processing cannot catch up
>>>>>>>>>> with the
>>>>>>>>>>
>>>>>>>>>> event rate
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> and
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    the expected timestamp events gets queued for
>>>>>>>>>> long
>>>>>>>>>>
>>>>>>>>>> time, this
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> approach
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>    might jump the gun.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> I believe the above approach and discussion goes
>>>>>>>>>> close to
>>>>>>>>>>
>>>>>>>>>> the approach
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> A.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> -----------
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> I like the idea of having an even count based
>>>>>>>>>> punctuate.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> -----------
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> I agree with the discussion around approach C,
>>>>>>>>>> that we
>>>>>>>>>>
>>>>>>>>>> should provide
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> the
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> user with the option to choose system time or
>>>>>>>>>> event time
>>>>>>>>>>
>>>>>>>>>> based
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> punctuates.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> But I believe that the user predominantly wants to
>>>>>>>>>> use
>>>>>>>>>>
>>>>>>>>>> event time while
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> not
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> missing out on regular punctuates due to event
>>>>>>>>>> delays or
>>>>>>>>>>
>>>>>>>>>> event
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> absences.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> Hence a complex punctuate option as Matthias
>>>>>>>>>> mentioned
>>>>>>>>>>
>>>>>>>>>> (quoted below)
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> would
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> be most apt.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> "- We might want to add "complex" schedules later
>>>>>>>>>> on
>>>>>>>>>>
>>>>>>>>>> (like, punctuate
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> on
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>>>>>>> time
>>>>>>>>>>
>>>>>>>>>> whatever comes
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> first)."
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> -----------
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> I think I read somewhere that Kafka Streams
>>>>>>>>>> started with
>>>>>>>>>>
>>>>>>>>>> System Time as
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> the
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> punctuation standard, but was later changed to
>>>>>>>>>> Event Time.
>>>>>>>>>>
>>>>>>>>>> I guess
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> there
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> would be some good reason behind it. As Kafka
>>>>>>>>>> Streams want
>>>>>>>>>>
>>>>>>>>>> to evolve
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> more
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> on the Stream Processing front, I believe the
>>>>>>>>>> emphasis on
>>>>>>>>>>
>>>>>>>>>> event time
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> would
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> remain quite strong.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> With Regards,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> Arun Mathew
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>>>>>>>
>>>>>>>>>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> wrote:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> Yeah I like PuncutationType much better; I just
>>>>>>>>>> threw
>>>>>>>>>>
>>>>>>>>>> Time out there
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>>>>>>> still
>>>>>>>>>>
>>>>>>>>>> think it's
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> worth considering what this buys us over an
>>>>>>>>>> additional
>>>>>>>>>>
>>>>>>>>>> callback. I
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> foresee a number of punctuate implementations
>>>>>>>>>> following
>>>>>>>>>>
>>>>>>>>>> this pattern:
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> public void punctuate(PunctuationType type) {
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>     switch (type) {
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>         case EVENT_TIME:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>             methodA();
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>             break;
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>         case SYSTEM_TIME:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>             methodB();
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>             break;
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>     }
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> }
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> I guess one advantage of this approach is we
>>>>>>>>>> could add
>>>>>>>>>>
>>>>>>>>>> additional
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> punctuation types later in a backwards compatible
>>>>>>>>>> way
>>>>>>>>>>
>>>>>>>>>> (like event
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> count
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> as you mentioned).
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> -Tommy
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>>>>>>> Sax wrote:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> That sounds promising.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> I am just wondering if `Time` is the best name.
>>>>>>>>>> Maybe we
>>>>>>>>>>
>>>>>>>>>> want to
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> add
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> other non-time based punctuations at some point
>>>>>>>>>> later. I
>>>>>>>>>>
>>>>>>>>>> would
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> suggest
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> enum PunctuationType {
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>   EVENT_TIME,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>   SYSTEM_TIME,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> }
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> or similar. Just to keep the door open -- it's
>>>>>>>>>> easier to
>>>>>>>>>>
>>>>>>>>>> add new
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> stuff
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> if the name is more generic.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> -Matthias
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> I agree that the framework providing and
>>>>>>>>>> managing the
>>>>>>>>>>
>>>>>>>>>> notion of
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> stream
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> time is valuable and not something we would
>>>>>>>>>> want to
>>>>>>>>>>
>>>>>>>>>> delegate to
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> the
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>>>>>>> separate
>>>>>>>>>>
>>>>>>>>>> callback
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> (option
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> C)
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> is that messy (it could just be a default
>>>>>>>>>> method with
>>>>>>>>>>
>>>>>>>>>> an empty
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> implementation), but if we wanted a single API
>>>>>>>>>> to
>>>>>>>>>>
>>>>>>>>>> handle both
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> cases,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> how about something like the following?
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> enum Time {
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>    STREAM,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>    CLOCK
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> }
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> Then on ProcessorContext:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>>>>>>> We could
>>>>>>>>>>
>>>>>>>>>> allow
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> this
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> to
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> be called once for each value of time to mix
>>>>>>>>>>
>>>>>>>>>> approaches.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> Then the Processor API becomes:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>>>>>>>
>>>>>>>>>> schedule resulted
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> in
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> this call.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> Thoughts?
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>>>>>>> Sax
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> I was thinking about the four options you
>>>>>>>>>> proposed in
>>>>>>>>>>
>>>>>>>>>> more
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> details
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> and
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> this are my thoughts:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> (A) You argue, that users can still
>>>>>>>>>> "punctuate" on
>>>>>>>>>>
>>>>>>>>>> event-time
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> via
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> process(), but I am not sure if this is
>>>>>>>>>> possible.
>>>>>>>>>>
>>>>>>>>>> Note, that
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> users
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> only
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> get record timestamps via context.timestamp().
>>>>>>>>>> Thus,
>>>>>>>>>>
>>>>>>>>>> users
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> would
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> need
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> to
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> track the time progress per partition (based
>>>>>>>>>> on the
>>>>>>>>>>
>>>>>>>>>> partitions
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> they
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> obverse via context.partition(). (This alone
>>>>>>>>>> puts a
>>>>>>>>>>
>>>>>>>>>> huge burden
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> on
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> the
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> user by itself.) However, users are not
>>>>>>>>>> notified at
>>>>>>>>>>
>>>>>>>>>> startup
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> what
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> partitions are assigned, and user are not
>>>>>>>>>> notified
>>>>>>>>>>
>>>>>>>>>> when
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> partitions
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> get
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> revoked. Because this information is not
>>>>>>>>>> available,
>>>>>>>>>>
>>>>>>>>>> it's not
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> possible
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> to
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> "manually advance" stream-time, and thus
>>>>>>>>>> event-time
>>>>>>>>>>
>>>>>>>>>> punctuation
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> within
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> process() seems not to be possible -- or do
>>>>>>>>>> you see a
>>>>>>>>>>
>>>>>>>>>> way to
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> get
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> it
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> done? And even if, it might still be too
>>>>>>>>>> clumsy to
>>>>>>>>>>
>>>>>>>>>> use.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> (B) This does not allow to mix both
>>>>>>>>>> approaches, thus
>>>>>>>>>>
>>>>>>>>>> limiting
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> what
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> users
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> can do.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> (C) This should give all flexibility we need.
>>>>>>>>>> However,
>>>>>>>>>>
>>>>>>>>>> just
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> adding
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> one
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> more method seems to be a solution that is too
>>>>>>>>>> simple
>>>>>>>>>>
>>>>>>>>>> (cf my
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> comments
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> below).
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>>>>>>> sure how
>>>>>>>>>>
>>>>>>>>>> a user
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> could
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> enable system-time and event-time punctuation
>>>>>>>>>> in
>>>>>>>>>>
>>>>>>>>>> parallel.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> Overall options (C) seems to be the most
>>>>>>>>>> promising
>>>>>>>>>>
>>>>>>>>>> approach to
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> me.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> Because I also favor a clean API, we might
>>>>>>>>>> keep
>>>>>>>>>>
>>>>>>>>>> current
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> punctuate()
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>>>>>>> at some
>>>>>>>>>>
>>>>>>>>>> later
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> point
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> when
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> people use the "new punctuate API".
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> Couple of follow up questions:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> - I am wondering, if we should have two
>>>>>>>>>> callback
>>>>>>>>>>
>>>>>>>>>> methods or
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> just
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> one
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> (ie, a unified for system and event time
>>>>>>>>>> punctuation
>>>>>>>>>>
>>>>>>>>>> or one for
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> each?).
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> - If we have one, how can the user figure out,
>>>>>>>>>> which
>>>>>>>>>>
>>>>>>>>>> condition
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> did
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> trigger?
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> - How would the API look like, for registering
>>>>>>>>>>
>>>>>>>>>> different
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> punctuate
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> - We might want to add "complex" schedules
>>>>>>>>>> later on
>>>>>>>>>>
>>>>>>>>>> (like,
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> punctuate
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> on
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>>>>>>> system-time
>>>>>>>>>>
>>>>>>>>>> whatever
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> comes
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> first). I don't say we should add this right
>>>>>>>>>> away, but
>>>>>>>>>>
>>>>>>>>>> we might
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> want
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> to
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> define the API in a way, that it allows
>>>>>>>>>> extensions
>>>>>>>>>>
>>>>>>>>>> like this
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> later
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> on,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> without redesigning the API (ie, the API
>>>>>>>>>> should be
>>>>>>>>>>
>>>>>>>>>> designed
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> extensible)
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> - Did you ever consider count-based
>>>>>>>>>> punctuation?
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> I understand, that you would like to solve a
>>>>>>>>>> simple
>>>>>>>>>>
>>>>>>>>>> problem,
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> but
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> we
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> learned from the past, that just "adding some
>>>>>>>>>> API"
>>>>>>>>>>
>>>>>>>>>> quickly
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> leads
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> to a
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> not very well defined API that needs time
>>>>>>>>>> consuming
>>>>>>>>>>
>>>>>>>>>> clean up
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> later on
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>>>>>>> holistic
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> punctuation
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> KIP
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> with this from the beginning on to avoid later
>>>>>>>>>> painful
>>>>>>>>>>      >
>>>>>>>>>>      >     >> redesign.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> -Matthias
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> Thanks Thomas,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> I'm also wary of changing the existing
>>>>>>>>>> semantics of
>>>>>>>>>>      >
>>>>>>>>>>      >     >> punctuate,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> for
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> backward compatibility reasons, although I
>>>>>>>>>> like the
>>>>>>>>>>      >
>>>>>>>>>>      >     >> conceptual
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> simplicity of that option.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>>>>>>> a way,
>>>>>>>>>>
>>>>>>>>>> uglier.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> I
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> added
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> this to the KIP now as option (C).
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>>>>>>> more
>>>>>>>>>>
>>>>>>>>>> flexible,
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> as
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> it
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> allows
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> you to return any value, you're not limited
>>>>>>>>>> to event
>>>>>>>>>>
>>>>>>>>>> time or
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> system
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> time
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> (although I don't see an actual use case
>>>>>>>>>> where you
>>>>>>>>>>
>>>>>>>>>> might need
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> anything
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> else then those two). Hence I also proposed
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> option to
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> allow
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> users
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>>>>>>> for
>>>>>>>>>>
>>>>>>>>>> them given
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> the
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> presence or absence of messages, much like
>>>>>>>>>> they can
>>>>>>>>>>
>>>>>>>>>> decide
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> what
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> msg
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> time
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>>>>>>> What do
>>>>>>>>>>
>>>>>>>>>> you
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> think
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> about
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> that? This is probably most flexible but also
>>>>>>>>>> most
>>>>>>>>>>      >
>>>>>>>>>>      >     >> complicated.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> All comments appreciated.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> Cheers,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> Michal
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> Although I fully agree we need a way to
>>>>>>>>>> trigger
>>>>>>>>>>
>>>>>>>>>> periodic
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> processing
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> that is independent from whether and when
>>>>>>>>>> messages
>>>>>>>>>>
>>>>>>>>>> arrive,
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> I'm
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> not sure
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> I like the idea of changing the existing
>>>>>>>>>> semantics
>>>>>>>>>>
>>>>>>>>>> across
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> the
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> board.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> What if we added an additional callback to
>>>>>>>>>> Processor
>>>>>>>>>>
>>>>>>>>>> that
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> can
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> be
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>>>>>>> always
>>>>>>>>>>
>>>>>>>>>> called at
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> fixed, wall
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>>>>>>> have to
>>>>>>>>>>
>>>>>>>>>> give
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> up
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> the
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> notion
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> of stream time to be able to do periodic
>>>>>>>>>> processing.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>>>>>>> Borowiecki
>>>>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> Hi all,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>>>>>>>
>>>>>>>>>> punctuate
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> semantics
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>>>>>>>
>>>>>>>>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/ confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>>>>>>>
>>>>>>>>>> 138%
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> 3A+C
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> hange+
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> punctuate+semantics>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> .
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> Appreciating there can be different views
>>>>>>>>>> on
>>>>>>>>>>
>>>>>>>>>> system-time
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> vs
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> event-
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> time
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>>>>>>> case and
>>>>>>>>>>
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> importance of
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>>>>>>> I've
>>>>>>>>>>
>>>>>>>>>> left it
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> quite
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> open
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> and
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>>>>>>>
>>>>>>>>>> progresses.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> Thanks,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>>>>> Michal
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> --
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>     Tommy Becker
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>     Senior Software Engineer
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>     tivo.com
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> ________________________________
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> This email and any attachments may contain
>>>>>>>>>> confidential
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> privileged material for the sole use of the
>>>>>>>>>> intended
>>>>>>>>>>
>>>>>>>>>> recipient.
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> Any
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> review, copying, or distribution of this email
>>>>>>>>>> (or any
>>>>>>>>>>      >
>>>>>>>>>>      >     >> attachments)
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> by others is prohibited. If you are not the
>>>>>>>>>> intended
>>>>>>>>>>
>>>>>>>>>> recipient,
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> please contact the sender immediately and
>>>>>>>>>> permanently
>>>>>>>>>>
>>>>>>>>>> delete this
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> email and any attachments. No employee or agent
>>>>>>>>>> of TiVo
>>>>>>>>>>
>>>>>>>>>> Inc. is
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> authorized to conclude any binding agreement on
>>>>>>>>>> behalf
>>>>>>>>>>
>>>>>>>>>> of TiVo
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> Inc.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>>>>>>> only be
>>>>>>>>>>
>>>>>>>>>> made by a
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>> signed written agreement.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> --
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>     Tommy Becker
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>     Senior Software Engineer
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>     tivo.com
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> ________________________________
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> This email and any attachments may contain
>>>>>>>>>> confidential
>>>>>>>>>>
>>>>>>>>>> and
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> privileged
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> material for the sole use of the intended
>>>>>>>>>> recipient. Any
>>>>>>>>>>
>>>>>>>>>> review,
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>> copying,
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> or distribution of this email (or any
>>>>>>>>>> attachments) by
>>>>>>>>>>
>>>>>>>>>> others is
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>> prohibited.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> If you are not the intended recipient, please
>>>>>>>>>> contact the
>>>>>>>>>>
>>>>>>>>>> sender
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> immediately and permanently delete this email and
>>>>>>>>>> any
>>>>>>>>>>
>>>>>>>>>> attachments. No
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>>>>>>> conclude
>>>>>>>>>>
>>>>>>>>>> any binding
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>>>>>>> Binding
>>>>>>>>>>
>>>>>>>>>> agreements with
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >> TiVo
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>> Inc. may only be made by a signed written
>>>>>>>>>> agreement.
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>>
>>>>>>>>>>      >
>>>>>>>>>>      >     >>
>>>>>>>>>>      >
>>>>>>>>>>      >     >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > --
>>>>>>>>>>      >
>>>>>>>>>>      > <http://www.openbet.com/> <http://www.openbet.com/>
>>>>>>>>>>
>>>>>>>>>>      >
>>>>>>>>>>      > *Michal Borowiecki*
>>>>>>>>>>      >
>>>>>>>>>>      > *Senior Software Engineer L4*
>>>>>>>>>>      >
>>>>>>>>>>      > *T: *
>>>>>>>>>>      >
>>>>>>>>>>      > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>>>>>>>      >
>>>>>>>>>>      > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > *E: *
>>>>>>>>>>      >
>>>>>>>>>>      > michal.borowiecki@openbet.com
>>>>>>>>>>      >
>>>>>>>>>>      > *W: *
>>>>>>>>>>      >
>>>>>>>>>>      > www.openbet.com
>>>>>>>>>>      >
>>>>>>>>>>      > *OpenBet Ltd*
>>>>>>>>>>      >
>>>>>>>>>>      > Chiswick Park Building 9
>>>>>>>>>>      >
>>>>>>>>>>      > 566 Chiswick High Rd
>>>>>>>>>>      >
>>>>>>>>>>      > London
>>>>>>>>>>      >
>>>>>>>>>>      > W4 5XT
>>>>>>>>>>      >
>>>>>>>>>>      > UK
>>>>>>>>>>      >
>>>>>>>>>>      > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      > This message is confidential and intended only for the
>>>>>>>>>> addressee.
>>>>>>>>>>
>>>>>>>>>> If you
>>>>>>>>>>
>>>>>>>>>>      > have received this message in error, please immediately
>>>>>>>>>> notify the
>>>>>>>>>>      > postmaster@openbet.com and delete it from your system as
>>>>>>>>>> well as
>>>>>>>>>>
>>>>>>>>>> any
>>>>>>>>>>
>>>>>>>>>>      > copies. The content of e-mails as well as traffic data may
>>>>>>>>>> be
>>>>>>>>>>
>>>>>>>>>> monitored by
>>>>>>>>>>
>>>>>>>>>>      > OpenBet for employment and security purposes. To protect
>>>>>>>>>> the
>>>>>>>>>>
>>>>>>>>>> environment
>>>>>>>>>>
>>>>>>>>>>      > please do not print this e-mail unless necessary. OpenBet
>>>>>>>>>> Ltd.
>>>>>>>>>>
>>>>>>>>>> Registered
>>>>>>>>>>
>>>>>>>>>>      > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>>>>>>> London,
>>>>>>>>>>
>>>>>>>>>> W4 5XT,
>>>>>>>>>>
>>>>>>>>>>      > United Kingdom. A company registered in England and Wales.
>>>>>>>>>>
>>>>>>>>>> Registered no.
>>>>>>>>>>
>>>>>>>>>>      > 3134634. VAT no. GB927523612
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>      >
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>      Tommy Becker
>>>>>>>>>>
>>>>>>>>>>      Senior Software Engineer
>>>>>>>>>>
>>>>>>>>>>      O +1 919.460.4747 <%28919%29%20460-4747>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>      tivo.com
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> ________________________________
>>>>>>>>>>
>>>>>>>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>> <http://www.openbet.com/> Michal Borowiecki
>>>>>>>>>> Senior Software Engineer L4
>>>>>>>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>> -- 
>>>>>>> Signature
>>>>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>>>>> Senior Software Engineer L4
>>>>>>> 	T: 	+44 208 742 1600
>>>>>>>
>>>>>>> 	
>>>>>>> 	+44 203 249 8448
>>>>>>>
>>>>>>> 	
>>>>>>> 	
>>>>>>> 	E: 	michal.borowiecki@openbet.com
>>>>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>>>>
>>>>>>> 	
>>>>>>> 	OpenBet Ltd
>>>>>>>
>>>>>>> 	Chiswick Park Building 9
>>>>>>>
>>>>>>> 	566 Chiswick High Rd
>>>>>>>
>>>>>>> 	London
>>>>>>>
>>>>>>> 	W4 5XT
>>>>>>>
>>>>>>> 	UK
>>>>>>>
>>>>>>> 	
>>>>>>> <https://www.openbet.com/email_promo>
>>>>>>>
>>>>>>> This message is confidential and intended only for the addressee. If
>>>>>>> you have received this message in error, please immediately notify the
>>>>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>>>>> from your system as well as any copies. The content of e-mails as well
>>>>>>> as traffic data may be monitored by OpenBet for employment and
>>>>>>> security purposes. To protect the environment please do not print this
>>>>>>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>>>>>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>>>>>> company registered in England and Wales. Registered no. 3134634. VAT
>>>>>>> no. GB927523612
>>>>>>>
>>>>>> -- 
>>>>>> Signature
>>>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>>>> Senior Software Engineer L4
>>>>>> 	T: 	+44 208 742 1600
>>>>>>
>>>>>> 	
>>>>>> 	+44 203 249 8448
>>>>>>
>>>>>> 	
>>>>>> 	
>>>>>> 	E: 	michal.borowiecki@openbet.com
>>>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>>>
>>>>>> 	
>>>>>> 	OpenBet Ltd
>>>>>>
>>>>>> 	Chiswick Park Building 9
>>>>>>
>>>>>> 	566 Chiswick High Rd
>>>>>>
>>>>>> 	London
>>>>>>
>>>>>> 	W4 5XT
>>>>>>
>>>>>> 	UK
>>>>>>
>>>>>> 	
>>>>>> <https://www.openbet.com/email_promo>
>>>>>>
>>>>>> This message is confidential and intended only for the addressee. If you
>>>>>> have received this message in error, please immediately notify the
>>>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>>>> from your system as well as any copies. The content of e-mails as well
>>>>>> as traffic data may be monitored by OpenBet for employment and security
>>>>>> purposes. To protect the environment please do not print this e-mail
>>>>>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>>>>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>>>>> registered in England and Wales. Registered no. 3134634. VAT no.
>>>>>> GB927523612
>>>>>>
>>>> -- 
>>>> Signature
>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>> Senior Software Engineer L4
>>>> 	T: 	+44 208 742 1600
>>>>
>>>> 	
>>>> 	+44 203 249 8448
>>>>
>>>> 	
>>>> 	
>>>> 	E: 	michal.borowiecki@openbet.com
>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>
>>>> 	
>>>> 	OpenBet Ltd
>>>>
>>>> 	Chiswick Park Building 9
>>>>
>>>> 	566 Chiswick High Rd
>>>>
>>>> 	London
>>>>
>>>> 	W4 5XT
>>>>
>>>> 	UK
>>>>
>>>> 	
>>>> <https://www.openbet.com/email_promo>
>>>>
>>>> This message is confidential and intended only for the addressee. If you
>>>> have received this message in error, please immediately notify the
>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>> from your system as well as any copies. The content of e-mails as well
>>>> as traffic data may be monitored by OpenBet for employment and security
>>>> purposes. To protect the environment please do not print this e-mail
>>>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>>> registered in England and Wales. Registered no. 3134634. VAT no.
>>>> GB927523612
>>>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If you
>> have received this message in error, please immediately notify the
>> postmaster@openbet.com <ma...@openbet.com> and delete it
>> from your system as well as any copies. The content of e-mails as well
>> as traffic data may be monitored by OpenBet for employment and security
>> purposes. To protect the environment please do not print this e-mail
>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>> registered in England and Wales. Registered no. 3134634. VAT no.
>> GB927523612
>>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
>> Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?

If you go with stream-time and don't have any input records for all
partitions, punctuate would not be called at all, and thus your
dashboard would "freeze".

>> I thought about cron-type things, but aren't they better triggered by an
>> external scheduler (they're more flexible anyway), which then feeds
>> "commands" into the topology?

I guess it depends what kind of periodic action you want to trigger. The
"cron job" was just an analogy. Maybe it's just a heartbeat to some
other service, that signals that your Streams app is still running.


-Matthias


On 4/24/17 10:02 AM, Michal Borowiecki wrote:
> Thanks!
> 
> Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?
> 
> Unless too frequent messages on replay could overpower it?
> 
> 
> I thought about cron-type things, but aren't they better triggered by an
> external scheduler (they're more flexible anyway), which then feeds
> "commands" into the topology?
> 
> Just my 2c.
> 
> Cheers,
> 
> Michal
> 
> 
> On 24/04/17 17:57, Matthias J. Sax wrote:
>> A simple example would be some dashboard app, that needs to get
>> "current" status in regular time intervals (ie, and real-time app).
>>
>> Or something like a "scheduler" -- think "cron job" application.
>>
>>
>> -Matthias
>>
>> On 4/24/17 2:23 AM, Michal Borowiecki wrote:
>>> Hi Matthias,
>>>
>>> I agree it's difficult to reason about the hybrid approach, I certainly
>>> found it hard and I'm totally on board with the mantra.
>>>
>>> I'd be happy to limit the scope of this KIP to add system-time
>>> punctuation semantics (in addition to existing stream-time semantics)
>>> and leave more complex schemes for users to implement on top of that.
>>>
>>> Further additional PunctuationTypes, could then be added by future KIPs,
>>> possibly including the hybrid approach once it has been given more thought.
>>>
>>>> There are real-time applications, that want to get
>>>> callbacks in regular system-time intervals (completely independent from
>>>> stream-time).
>>> Can you please describe what they are, so that I can put them on the
>>> wiki for later reference?
>>>
>>> Thanks,
>>>
>>> Michal
>>>
>>>
>>> On 23/04/17 21:27, Matthias J. Sax wrote:
>>>> Hi,
>>>>
>>>> I do like Damian's API proposal about the punctuation callback function.
>>>>
>>>> I also did reread the KIP and thought about the semantics we want to
>>>> provide.
>>>>
>>>>> Given the above, I don't see a reason any more for a separate system-time based punctuation.
>>>> I disagree here. There are real-time applications, that want to get
>>>> callbacks in regular system-time intervals (completely independent from
>>>> stream-time). Thus we should allow this -- if we really follow the
>>>> "hybrid" approach, this could be configured with stream-time interval
>>>> infinite and delay whatever system-time punctuation interval you want to
>>>> have. However, I would like to add a proper API for this and do this
>>>> configuration under the hood (that would allow one implementation within
>>>> all kind of branching for different cases).
>>>>
>>>> Thus, we definitely should have PunctutionType#StreamTime and
>>>> #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
>>>> a fan of your latest API proposal.
>>>>
>>>>
>>>> About the hybrid approach in general. On the one hand I like it, on the
>>>> other hand, it seems to be rather (1) complicated (not necessarily from
>>>> an implementation point of view, but for people to understand it) and
>>>> (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
>>>>
>>>>> It may appear complicated at first but I do think these semantics will
>>>>> still be more understandable to users than having 2 separate punctuation
>>>>> schedules/callbacks with different PunctuationTypes.
>>>> This statement only holds if you apply strong assumptions that I don't
>>>> believe hold in general -- see (2) for details -- and I think it is
>>>> harder than you assume to reason about the hybrid approach in general.
>>>> IMHO, the hybrid approach is a "false friend" that seems to be easy to
>>>> reason about...
>>>>
>>>>
>>>> (1) Streams always embraced "easy to use" and we should really be
>>>> careful to keep it this way. On the other hand, as we are talking about
>>>> changes to PAPI, it won't affect DSL users (DSL does not use punctuation
>>>> at all at the moment), and thus, the "easy to use" mantra might not be
>>>> affected, while it will allow advanced users to express more complex stuff.
>>>>
>>>> I like the mantra: "make simple thing easy and complex things possible".
>>>>
>>>> (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
>>>> implicit assumption that even-time progresses at the same "speed" as
>>>> system-time during regular processing. This implies the assumption that
>>>> a slower progress in stream-time indicates the absence of input events
>>>> (and that later arriving input events will have a larger event-time with
>>>> high probability). Even if this might be true for some use cases, I
>>>> doubt it holds in general. Assume that you get a spike in traffic and
>>>> for some reason stream-time does advance slowly because you have more
>>>> records to process. This might trigger a system-time based punctuation
>>>> call even if this seems not to be intended. I strongly believe that it
>>>> is not easy to reason about the semantics of the hybrid approach (even
>>>> if the intentional semantics would be super useful -- but I doubt that
>>>> we get want we ask for).
>>>>
>>>> Thus, I also believe that one might need different "configuration"
>>>> values for the hybrid approach if you run the same code for different
>>>> scenarios: regular processing, re-processing, catching up scenario. And
>>>> as the term "configuration" implies, we might be better off to not mix
>>>> configuration with business logic that is expressed via code.
>>>>
>>>>
>>>> One more comment: I also don't think that the hybrid approach is
>>>> deterministic as claimed in the use-case subpage. I understand the
>>>> reasoning and agree, that it is deterministic if certain assumptions
>>>> hold -- compare above -- and if configured correctly. But strictly
>>>> speaking it's not because there is a dependency on system-time (and
>>>> IMHO, if system-time is involved it cannot be deterministic by definition).
>>>>
>>>>
>>>>> I see how in theory this could be implemented on top of the 2 punctuate
>>>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>>>> the other system-time based) but it would be a much more complicated
>>>>> scheme and I don't want to suggest that.
>>>> I agree that expressing the intended hybrid semantics is harder if we
>>>> offer only #StreamTime and #SystemTime punctuation. However, I also
>>>> believe that the hybrid approach is a "false friend" with regard to
>>>> reasoning about the semantics (it indicates that it more easy as it is
>>>> in reality). Therefore, we might be better off to not offer the hybrid
>>>> approach and make it clear to a developed, that it is hard to mix
>>>> #StreamTime and #SystemTime in a semantically sound way.
>>>>
>>>>
>>>> Looking forward to your feedback. :)
>>>>
>>>> -Matthias
>>>>
>>>>
>>>>
>>>>
>>>> On 4/22/17 11:43 AM, Michal Borowiecki wrote:
>>>>> Hi all,
>>>>>
>>>>> Looking for feedback on the functional interface approach Damian
>>>>> proposed. What do people think?
>>>>>
>>>>> Further on the semantics of triggering punctuate though:
>>>>>
>>>>> I ran through the 2 use cases that Arun had kindly put on the wiki
>>>>> (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
>>>>> in my head and on a whiteboard and I can't find a better solution than
>>>>> the "hybrid" approach he had proposed.
>>>>>
>>>>> I see how in theory this could be implemented on top of the 2 punctuate
>>>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>>>> the other system-time based) but it would be a much more complicated
>>>>> scheme and I don't want to suggest that.
>>>>>
>>>>> However, to add to the hybrid algorithm proposed, I suggest one
>>>>> parameter to that: a tolerance period, expressed in milliseconds
>>>>> system-time, after which the punctuation will be invoked in case the
>>>>> stream-time advance hasn't triggered it within the requested interval
>>>>> since the last invocation of punctuate (as recorded in system-time)
>>>>>
>>>>> This would allow a user-defined tolerance for late arriving events. The
>>>>> trade off would be left for the user to decide: regular punctuation in
>>>>> the case of absence of events vs allowing for records arriving late or
>>>>> some build-up due to processing not catching up with the event rate.
>>>>> In the one extreme, this tolerance could be set to infinity, turning
>>>>> hybrid into simply stream-time based punctuate, like we have now. In the
>>>>> other extreme, the tolerance could be set to 0, resulting in a
>>>>> system-time upper bound on the effective punctuation interval.
>>>>>
>>>>> Given the above, I don't see a reason any more for a separate
>>>>> system-time based punctuation. The "hybrid" approach with 0ms tolerance
>>>>> would under normal operation trigger at regular intervals wrt the
>>>>> system-time, except in cases of re-play/catch-up, where the stream time
>>>>> advances faster than system time. In these cases punctuate would happen
>>>>> more often than the specified interval wrt system time. However, the
>>>>> use-cases that need system-time punctuations (that I've seen at least)
>>>>> really only have a need for an upper bound on punctuation delay but
>>>>> don't need a lower bound.
>>>>>
>>>>> To that effect I'd propose the api to be as follows, on ProcessorContext:
>>>>>
>>>>> schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
>>>>>
>>>>> schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
>>>>>
>>>>> Punctuation is triggered when either:
>>>>> - the stream time advances past the (stream time of the previous
>>>>> punctuation) + interval;
>>>>> - or (iff the toleranceInterval is set) when the system time advances
>>>>> past the (system time of the previous punctuation) + interval +
>>>>> toleranceInterval
>>>>>
>>>>> In either case:
>>>>> - we trigger punctuate passing as the argument the stream time at which
>>>>> the current punctuation was meant to happen
>>>>> - next punctuate is scheduled at (stream time at which the current
>>>>> punctuation was meant to happen) + interval
>>>>>
>>>>> It may appear complicated at first but I do think these semantics will
>>>>> still be more understandable to users than having 2 separate punctuation
>>>>> schedules/callbacks with different PunctuationTypes.
>>>>>
>>>>>
>>>>>
>>>>> PS. Having re-read this, maybe the following alternative would be easier
>>>>> to understand (WDYT?):
>>>>>
>>>>> schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
>>>>>
>>>>> schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
>>>>>
>>>>> Punctuation is triggered when either:
>>>>> - the stream time advances past the (stream time of the previous
>>>>> punctuation) + streamTimeInterval;
>>>>> - or (iff systemTimeUpperBound is set) when the system time advances
>>>>> past the (system time of the previous punctuation) + systemTimeUpperBound
>>>>>
>>>>> Awaiting comments.
>>>>>
>>>>> Thanks,
>>>>> Michal
>>>>>
>>>>> On 21/04/17 16:56, Michal Borowiecki wrote:
>>>>>> Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>>>>>> in favour of something that doesn't return a record. Not a problem though.
>>>>>>
>>>>>>
>>>>>> On 21/04/17 16:32, Damian Guy wrote:
>>>>>>> Thanks Michal,
>>>>>>> I agree Transformer.punctuate should also be void, but we can deprecate
>>>>>>> that too in favor of the new interface.
>>>>>>>
>>>>>>> Thanks for the javadoc PR!
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Damian
>>>>>>>
>>>>>>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>>
>>>>>>>> Yes, that looks better to me.
>>>>>>>>
>>>>>>>> Note that punctuate on Transformer is currently returning a record, but I
>>>>>>>> think it's ok to have all output records be sent via
>>>>>>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>>>>>>> multiple records from one invocation of punctuate.
>>>>>>>>
>>>>>>>> This way it's consistent between Processor and Transformer.
>>>>>>>>
>>>>>>>>
>>>>>>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>>>>>>> there:
>>>>>>>>
>>>>>>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>>>>>>
>>>>>>>> and PR: https://github.com/apache/kafka/pull/2884
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>>
>>>>>>>> Michal
>>>>>>>> On 20/04/17 18:55, Damian Guy wrote:
>>>>>>>>
>>>>>>>> Hi Michal,
>>>>>>>>
>>>>>>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>>>>>>> the API.
>>>>>>>> 1. deprecate the punctuate method on Processor
>>>>>>>> 2. create a new Functional Interface just for Punctuation, something like:
>>>>>>>> interface Punctuator {
>>>>>>>>     void punctuate(long timestamp)
>>>>>>>> }
>>>>>>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>>>>>>> interval, PunctuationType type, Punctuator callback)
>>>>>>>> 4. deprecate the existing schedule function
>>>>>>>>
>>>>>>>> Thoughts?
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Damian
>>>>>>>>
>>>>>>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>>>
>>>>>>>>> Hi Thomas,
>>>>>>>>>
>>>>>>>>> I would say our use cases fall in the same category as yours.
>>>>>>>>>
>>>>>>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>>>>>>
>>>>>>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>>>>>>> of design:
>>>>>>>>>
>>>>>>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>>>>>>
>>>>>>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>>>>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>>>>>>
>>>>>>>>> Since the output field X is derived in some non-trivial way, we don't
>>>>>>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>>>>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>>>>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>>>>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>>>>>>> another input field Z).
>>>>>>>>>
>>>>>>>>> So we have kv stores with the records and an additional kv store with
>>>>>>>>> timestamp->id mapping which act like an index where we periodically do a
>>>>>>>>> ranged query.
>>>>>>>>>
>>>>>>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>>>>>>> work when there were no regular msgs on the input topic.
>>>>>>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>>>>>>> outside we created a "ticker" that produced msgs once per second onto
>>>>>>>>> another topic and fed it into the same topology to trigger punctuate.
>>>>>>>>> This didn't work either, which was much more surprising to us at the
>>>>>>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>>>>>>> *all* input partitions receive messages regularly.
>>>>>>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>>>>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>>>>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>>>>>>> is consumed by the second topology and is its only input topic. There's a
>>>>>>>>> transformer on that topic which populates and updates the time-based
>>>>>>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>>>>>>> elapsed, the record id is sent to the main transformer, which
>>>>>>>>> updates/deletes the record from the main kv store and forwards the
>>>>>>>>> transformed record to the output topic.
>>>>>>>>>
>>>>>>>>> To me this setup feels horrendously complicated for what it does.
>>>>>>>>>
>>>>>>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>>>>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>>>>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>>>>>>> However, the ticks don't feel so hacky when you realise they give you
>>>>>>>>> some hypothetical benefits in predictability. You can reprocess the
>>>>>>>>> messages in a reproducible manner, since the topologies use event-time,
>>>>>>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>>>>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>>>>>>> reprocessing).
>>>>>>>>> To make that work though, we would have to have the stream time advance
>>>>>>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>>>>>>> presence of messages on the other input topic.
>>>>>>>>>
>>>>>>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>>>>>>> and the hybrid would work to simplify this a lot.
>>>>>>>>>
>>>>>>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>>>>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>>>>>>> have to go with punctuate there too.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Michal
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>>>>>>
>>>>>>>>> Here's an example that we currently have.  We have a streams processor
>>>>>>>>> that does a transform from one topic into another. One of the fields in
>>>>>>>>> the source topic record is an expiration time, and one of the functions
>>>>>>>>> of the processor is to ensure that expired records get deleted promptly
>>>>>>>>> after that time passes (typically days or weeks after the message was
>>>>>>>>> originally produced). To do that, the processor keeps a state store of
>>>>>>>>> keys and expiration times, iterates that store in punctuate(), and
>>>>>>>>> emits delete (null) records for expired items. This needs to happen at
>>>>>>>>> some minimum interval regardless of the incoming message rate of the
>>>>>>>>> source topic.
>>>>>>>>>
>>>>>>>>> In this scenario, the expiration of records is the primary function of
>>>>>>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>>>>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>>>>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>>>>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>>>>>>> range of use-cases.
>>>>>>>>>
>>>>>>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>>>>>>
>>>>>>>>> I apologize for the longer email below.  To my defense, it started
>>>>>>>>> out much
>>>>>>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>>>>>>> devil's
>>>>>>>>> advocate for a number of arguments brought forth in order to help
>>>>>>>>> improve
>>>>>>>>> this KIP -- I am not implying I necessarily disagree with the
>>>>>>>>> arguments.
>>>>>>>>>
>>>>>>>>> That aside, here are some further thoughts.
>>>>>>>>>
>>>>>>>>> First, there are (at least?) two categories for actions/behavior you
>>>>>>>>> invoke
>>>>>>>>> via punctuate():
>>>>>>>>>
>>>>>>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>>>>>>> to
>>>>>>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>>>>>>> the
>>>>>>>>> impact of punctuate is typically not observable by other processing
>>>>>>>>> nodes
>>>>>>>>> in the topology.
>>>>>>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>>>>>>> the
>>>>>>>>> punctuate is all about being observable by downstream processing
>>>>>>>>> nodes.
>>>>>>>>>
>>>>>>>>> A few releases back, we introduced record caches (DSL) and state
>>>>>>>>> store
>>>>>>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>>>>>>> relating to
>>>>>>>>> (2) where some users needed to control -- here: limit -- the
>>>>>>>>> downstream
>>>>>>>>> output rate of Kafka Streams because the downstream systems/apps
>>>>>>>>> would not
>>>>>>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>>>>>>> their
>>>>>>>>> scalability).  The argument for KIP-63, which notably did not
>>>>>>>>> introduce a
>>>>>>>>> "trigger" API, was that such an interaction with downstream systems
>>>>>>>>> is an
>>>>>>>>> operational concern;  it should not impact the processing *logic* of
>>>>>>>>> your
>>>>>>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>>>>>>> API,
>>>>>>>>> especially not the declarative DSL, with such operational concerns.
>>>>>>>>>
>>>>>>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>>>>>>> sorry, I
>>>>>>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>>>>>>> observing that our conversation is moving more and more into the
>>>>>>>>> direction
>>>>>>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>>>>>>> for
>>>>>>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>>>>>>> comments voiced here are about sth like "IF stream-time didn't
>>>>>>>>> trigger
>>>>>>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>>>>>>> want
>>>>>>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>>>>>>> related
>>>>>>>>> note, whatever we are discussing here will impact state store caches
>>>>>>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>>>>>>> should
>>>>>>>>> clarify any such impact here.
>>>>>>>>>
>>>>>>>>> Switching topics slightly.
>>>>>>>>>
>>>>>>>>> Jay wrote:
>>>>>>>>>
>>>>>>>>> One thing I've always found super important for this kind of design
>>>>>>>>> work
>>>>>>>>> is to do a really good job of cataloging the landscape of use cases
>>>>>>>>> and
>>>>>>>>> how prevalent each one is.
>>>>>>>>>
>>>>>>>>> +1 to this, as others have already said.
>>>>>>>>>
>>>>>>>>> Here, let me highlight -- just in case -- that when we talked about
>>>>>>>>> windowing use cases in the recent emails, the Processor API (where
>>>>>>>>> `punctuate` resides) does not have any notion of windowing at
>>>>>>>>> all.  If you
>>>>>>>>> want to do windowing *in the Processor API*, you must do so manually
>>>>>>>>> in
>>>>>>>>> combination with window stores.  For this reason I'd suggest to
>>>>>>>>> discuss use
>>>>>>>>> cases not just in general, but also in view of how you'd do so in the
>>>>>>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>>>>>>> `punctuate`
>>>>>>>>> does not impact the DSL at all, unless we add new functionality to
>>>>>>>>> it.
>>>>>>>>>
>>>>>>>>> Jay wrote in his strawman example:
>>>>>>>>>
>>>>>>>>> You aggregate click and impression data for a reddit like site.
>>>>>>>>> Every ten
>>>>>>>>> minutes you want to output a ranked list of the top 10 articles
>>>>>>>>> ranked by
>>>>>>>>> clicks/impressions for each geographical area. I want to be able
>>>>>>>>> run this
>>>>>>>>> in steady state as well as rerun to regenerate results (or catch up
>>>>>>>>> if it
>>>>>>>>> crashes).
>>>>>>>>>
>>>>>>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>>>>>>> we
>>>>>>>>> argued that the reason for saying "every ten minutes" above is not
>>>>>>>>> necessarily about because you want to output data *exactly* after ten
>>>>>>>>> minutes, but that you want to perform an aggregation based on 10-
>>>>>>>>> minute
>>>>>>>>> windows of input data; i.e., the point is about specifying the input
>>>>>>>>> for
>>>>>>>>> your aggregation, not or less about when the results of the
>>>>>>>>> aggregation
>>>>>>>>> should be send downstream.  To take an extreme example, you could
>>>>>>>>> disable
>>>>>>>>> record caches and let your app output a downstream update for every
>>>>>>>>> incoming input record.  If the last input record was from at minute 7
>>>>>>>>> of 10
>>>>>>>>> (for a 10-min window), then what your app would output at minute 10
>>>>>>>>> would
>>>>>>>>> be identical to what it had already emitted at minute 7 earlier
>>>>>>>>> anyways.
>>>>>>>>> This is particularly true when we take late-arriving data into
>>>>>>>>> account:  if
>>>>>>>>> a late record arrived at minute 13, your app would (by default) send
>>>>>>>>> a new
>>>>>>>>> update downstream, even though the "original" 10 minutes have already
>>>>>>>>> passed.
>>>>>>>>>
>>>>>>>>> Jay wrote...:
>>>>>>>>>
>>>>>>>>> There are a couple of tricky things that seem to make this hard
>>>>>>>>> with
>>>>>>>>>
>>>>>>>>> either
>>>>>>>>>
>>>>>>>>> of the options proposed:
>>>>>>>>> 1. If I emit this data using event time I have the problem
>>>>>>>>> described where
>>>>>>>>> a geographical region with no new clicks or impressions will fail
>>>>>>>>> to
>>>>>>>>>
>>>>>>>>> output
>>>>>>>>>
>>>>>>>>> results.
>>>>>>>>>
>>>>>>>>> ...and Arun Mathew wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> We window by the event time, but trigger punctuate in <punctuate
>>>>>>>>> interval>
>>>>>>>>> duration of system time, in the absence of an event crossing the
>>>>>>>>> punctuate
>>>>>>>>> event time.
>>>>>>>>>
>>>>>>>>> So, given what I wrote above about the status quo and what you can
>>>>>>>>> already
>>>>>>>>> do with it, is the concern that the state store cache doesn't give
>>>>>>>>> you
>>>>>>>>> *direct* control over "forcing an output after no later than X
>>>>>>>>> seconds [of
>>>>>>>>> processing-time]" but only indirect control through a cache
>>>>>>>>> size?  (Note
>>>>>>>>> that I am not dismissing the claims why this might be helpful.)
>>>>>>>>>
>>>>>>>>> Arun Mathew wrote:
>>>>>>>>>
>>>>>>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>>>>>>> output the
>>>>>>>>> event counts on each topic on each cluster aggregated over a 1
>>>>>>>>> minute
>>>>>>>>> window. We have to use event time to be able to cross check the
>>>>>>>>> counts.
>>>>>>>>>
>>>>>>>>> But
>>>>>>>>>
>>>>>>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>>>>>>> time in
>>>>>>>>>
>>>>>>>>> the
>>>>>>>>>
>>>>>>>>> absence of events. Otherwise the event counts for unexpired windows
>>>>>>>>> would
>>>>>>>>> be 0 which is bad.
>>>>>>>>>
>>>>>>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>>>>>>> absence
>>>>>>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>>>>>>> Scala
>>>>>>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>>>>>>> root
>>>>>>>>> cause that the downstream system interprets the absence of output in
>>>>>>>>> a
>>>>>>>>> particular way ("No output after 1 minute = I consider the output to
>>>>>>>>> be
>>>>>>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>>>>>>> possible)
>>>>>>>>> to correctly handle the difference between absence of output vs.
>>>>>>>>> output of
>>>>>>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>>>>>>> but
>>>>>>>>> want to understand the motivation better. :-)
>>>>>>>>>
>>>>>>>>> Also, to add some perspective, in some related discussions we talked
>>>>>>>>> about
>>>>>>>>> how a Kafka Streams application should not worry or not be coupled
>>>>>>>>> unnecessarily with such interpretation specifics in a downstream
>>>>>>>>> system's
>>>>>>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>>>>>>> more
>>>>>>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>>>>>>> than
>>>>>>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>>>>>>> and
>>>>>>>>> downstream systems, including helping to reconcile the differences in
>>>>>>>>> how
>>>>>>>>> these systems interpret changes, updates, late-arriving data,
>>>>>>>>> etc.  Kafka
>>>>>>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>>>>>>> processing from the specifics of downstream systems, thanks to
>>>>>>>>> specific
>>>>>>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>>>>>>> this
>>>>>>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>>>>>>> but
>>>>>>>>> it's currently awkward to use Connect for this", this might be a
>>>>>>>>> problem we
>>>>>>>>> can solve, too.)
>>>>>>>>>
>>>>>>>>> Switching topics slightly again.
>>>>>>>>>
>>>>>>>>> Thomas wrote:
>>>>>>>>>
>>>>>>>>> I'm not entirely convinced that a separate callback (option C)
>>>>>>>>> is that messy (it could just be a default method with an empty
>>>>>>>>> implementation), but if we wanted a single API to handle both
>>>>>>>>> cases,
>>>>>>>>> how about something like the following?
>>>>>>>>>
>>>>>>>>> enum Time {
>>>>>>>>>    STREAM,
>>>>>>>>>    CLOCK
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>>>>>>> then
>>>>>>>>> whatever the user is doing inside this method is a black box to Kafka
>>>>>>>>> Streams (similar to how we have no idea what the user does inside a
>>>>>>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>>>>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>>>>>>> won't
>>>>>>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>>>>>>> frequency than the processing-time action.  Or, we won't know whether
>>>>>>>>> the
>>>>>>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>>>>>>> minute of
>>>>>>>>> stream-time or 1-minute of processing-time, whichever comes
>>>>>>>>> first").  That
>>>>>>>>> said, I am not certain yet whether we would need such knowledge
>>>>>>>>> because,
>>>>>>>>> when using the Processor API, most of the work and decisions must be
>>>>>>>>> done
>>>>>>>>> by the user anyways.  It would matter though if the concept of
>>>>>>>>> "triggers"
>>>>>>>>> were to bubble up into the DSL because in the DSL the management of
>>>>>>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>>>>>>> Streams.
>>>>>>>>>
>>>>>>>>> [In any case, btw, we have the corner case where the user configured
>>>>>>>>> the
>>>>>>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>>>>>>> extractor), at which point both punctuate variants are based on the
>>>>>>>>> same
>>>>>>>>> time semantics / timeline.]
>>>>>>>>>
>>>>>>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>>>>>>> it
>>>>>>>>> this far. :-)
>>>>>>>>>
>>>>>>>>> More than happy to hear your thoughts!
>>>>>>>>> Michael
>>>>>>>>>
>>>>>>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Thanks Matthias.
>>>>>>>>> Sure, will correct it right away.
>>>>>>>>>
>>>>>>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Thanks for preparing this page!
>>>>>>>>>
>>>>>>>>> About terminology:
>>>>>>>>>
>>>>>>>>> You introduce the term "event time" -- but we should call this
>>>>>>>>> "stream
>>>>>>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>>>>>>> this
>>>>>>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>>>>>>
>>>>>>>>> Does this make sense to you?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>>>>>
>>>>>>>>> Thanks Ewen.
>>>>>>>>>
>>>>>>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>>>>>>
>>>>>>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>>>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>>>>>>> public.
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> Arun Mathew
>>>>>>>>>
>>>>>>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>     Arun,
>>>>>>>>>
>>>>>>>>>     I've given you permission to edit the wiki. Let me know if
>>>>>>>>> you run
>>>>>>>>>
>>>>>>>>> into any
>>>>>>>>>
>>>>>>>>>     issues.
>>>>>>>>>
>>>>>>>>>     -Ewen
>>>>>>>>>
>>>>>>>>>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp> <am...@yahoo-corp.jp>
>>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>     > Thanks Michal. I don’t have the access yet [arunmathew88].
>>>>>>>>> Should I
>>>>>>>>>
>>>>>>>>> be
>>>>>>>>>
>>>>>>>>>     > sending a separate mail for this?
>>>>>>>>>     >
>>>>>>>>>     > I thought one of the person following this thread would be
>>>>>>>>> able to
>>>>>>>>>
>>>>>>>>> give me
>>>>>>>>>
>>>>>>>>>     > access.
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>>>>>>>>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>>>     > *Date: *Friday, April 7, 2017 at 17:16
>>>>>>>>>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>>>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>>>>>> semantics
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > Hi Arun,
>>>>>>>>>     >
>>>>>>>>>     > I was thinking along the same lines as you, listing the use
>>>>>>>>> cases
>>>>>>>>>
>>>>>>>>> on the
>>>>>>>>>
>>>>>>>>>     > wiki, but didn't find time to get around doing that yet.
>>>>>>>>>     > Don't mind if you do it if you have access now.
>>>>>>>>>     > I was thinking it would be nice if, once we have the use
>>>>>>>>> cases
>>>>>>>>>
>>>>>>>>> listed,
>>>>>>>>>
>>>>>>>>>     > people could use likes to up-vote the use cases similar to
>>>>>>>>> what
>>>>>>>>>
>>>>>>>>> they're
>>>>>>>>>
>>>>>>>>>     > working on.
>>>>>>>>>     >
>>>>>>>>>     > I should have a bit more time to action this in the next
>>>>>>>>> few days,
>>>>>>>>>
>>>>>>>>> but
>>>>>>>>>
>>>>>>>>>     > happy for you to do it if you can beat me to it ;-)
>>>>>>>>>     >
>>>>>>>>>     > Cheers,
>>>>>>>>>     > Michal
>>>>>>>>>     >
>>>>>>>>>     > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>>>>>     >
>>>>>>>>>     > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > Of course. I was thinking of a subpage where people can
>>>>>>>>>
>>>>>>>>> collaborate.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > Will do as per Michael’s suggestion.
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > Regards,
>>>>>>>>>     >
>>>>>>>>>     > Arun Mathew
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>>>> <
>>>>>>>>>
>>>>>>>>> matthias@confluent.io> wrote:
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >     Please share your Wiki-ID and a committer can give you
>>>>>>>>> write
>>>>>>>>>
>>>>>>>>> access.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >     Btw: as you did not initiate the KIP, you should not
>>>>>>>>> change the
>>>>>>>>>
>>>>>>>>> KIP
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     without the permission of the original author -- in
>>>>>>>>> this case
>>>>>>>>>
>>>>>>>>> Michael.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >     So you might also just share your thought over the
>>>>>>>>> mailing list
>>>>>>>>>
>>>>>>>>> and
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     Michael can update the KIP page. Or, as an alternative,
>>>>>>>>> just
>>>>>>>>>
>>>>>>>>> create a
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     subpage for the KIP page.
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >     @Michael: WDYT?
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >     -Matthias
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>>>>>     >
>>>>>>>>>     >     > Hi Jay,
>>>>>>>>>     >
>>>>>>>>>     >     >           Thanks for the advise, I would like to list
>>>>>>>>> down
>>>>>>>>>
>>>>>>>>> the use cases as
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > per your suggestion. But it seems I don't have write
>>>>>>>>>
>>>>>>>>> permission to the
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > Apache Kafka Confluent Space. Whom shall I request
>>>>>>>>> for it?
>>>>>>>>>     >
>>>>>>>>>     >     >
>>>>>>>>>     >
>>>>>>>>>     >     > Regarding your last question. We are using a patch in
>>>>>>>>> our
>>>>>>>>>
>>>>>>>>> production system
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > which does exactly this.
>>>>>>>>>     >
>>>>>>>>>     >     > We window by the event time, but trigger punctuate in
>>>>>>>>>
>>>>>>>>> <punctuate interval>
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > duration of system time, in the absence of an event
>>>>>>>>> crossing
>>>>>>>>>
>>>>>>>>> the punctuate
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > event time.
>>>>>>>>>     >
>>>>>>>>>     >     >
>>>>>>>>>     >
>>>>>>>>>     >     > We are using Kafka Stream for our Audit Trail, where
>>>>>>>>> we need
>>>>>>>>>
>>>>>>>>> to output the
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > event counts on each topic on each cluster aggregated
>>>>>>>>> over a
>>>>>>>>>
>>>>>>>>> 1 minute
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > window. We have to use event time to be able to cross
>>>>>>>>> check
>>>>>>>>>
>>>>>>>>> the counts. But
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > we need to trigger punctuate [aggregate event pushes]
>>>>>>>>> by
>>>>>>>>>
>>>>>>>>> system time in the
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > absence of events. Otherwise the event counts for
>>>>>>>>> unexpired
>>>>>>>>>
>>>>>>>>> windows would
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > be 0 which is bad.
>>>>>>>>>     >
>>>>>>>>>     >     >
>>>>>>>>>     >
>>>>>>>>>     >     > "Maybe a hybrid solution works: I window by event
>>>>>>>>> time but
>>>>>>>>>
>>>>>>>>> trigger results
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > by system time for windows that have updated? Not
>>>>>>>>> really sure
>>>>>>>>>
>>>>>>>>> the details
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > of making that work. Does that work? Are there
>>>>>>>>> concrete
>>>>>>>>>
>>>>>>>>> examples where you
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > actually want the current behavior?"
>>>>>>>>>     >
>>>>>>>>>     >     >
>>>>>>>>>     >
>>>>>>>>>     >     > --
>>>>>>>>>     >
>>>>>>>>>     >     > With Regards,
>>>>>>>>>     >
>>>>>>>>>     >     >
>>>>>>>>>     >
>>>>>>>>>     >     > Arun Mathew
>>>>>>>>>     >
>>>>>>>>>     >     > Yahoo! JAPAN Corporation
>>>>>>>>>     >
>>>>>>>>>     >     >
>>>>>>>>>     >
>>>>>>>>>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>>>>>>
>>>>>>>>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >
>>>>>>>>>     >
>>>>>>>>>     >     >> Hi Jay,
>>>>>>>>>     >
>>>>>>>>>     >     >>
>>>>>>>>>     >
>>>>>>>>>     >     >> The hybrid solution is exactly what I expect and
>>>>>>>>> need for
>>>>>>>>>
>>>>>>>>> our use cases
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> when dealing with telecom data.
>>>>>>>>>     >
>>>>>>>>>     >     >>
>>>>>>>>>     >
>>>>>>>>>     >     >> Thanks
>>>>>>>>>     >
>>>>>>>>>     >     >> Tianji
>>>>>>>>>     >
>>>>>>>>>     >     >>
>>>>>>>>>     >
>>>>>>>>>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>>>>>>
>>>>>>>>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>
>>>>>>>>>     >
>>>>>>>>>     >     >>> Hey guys,
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> One thing I've always found super important for
>>>>>>>>> this kind
>>>>>>>>>
>>>>>>>>> of design work
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> is
>>>>>>>>>     >
>>>>>>>>>     >     >>> to do a really good job of cataloging the landscape
>>>>>>>>> of use
>>>>>>>>>
>>>>>>>>> cases and how
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> prevalent each one is. By that I mean not just
>>>>>>>>> listing lots
>>>>>>>>>
>>>>>>>>> of uses, but
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> also grouping them into categories that
>>>>>>>>> functionally need
>>>>>>>>>
>>>>>>>>> the same thing.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> In the absence of this it is very hard to reason
>>>>>>>>> about
>>>>>>>>>
>>>>>>>>> design proposals.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> From the proposals so far I think we have a lot of
>>>>>>>>>
>>>>>>>>> discussion around
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> possible apis, but less around what the user needs
>>>>>>>>> for
>>>>>>>>>
>>>>>>>>> different use
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> cases
>>>>>>>>>     >
>>>>>>>>>     >     >>> and how they would implement that using the api.
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> Here is an example:
>>>>>>>>>     >
>>>>>>>>>     >     >>> You aggregate click and impression data for a
>>>>>>>>> reddit like
>>>>>>>>>
>>>>>>>>> site. Every ten
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> minutes you want to output a ranked list of the top
>>>>>>>>> 10
>>>>>>>>>
>>>>>>>>> articles ranked by
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> clicks/impressions for each geographical area. I
>>>>>>>>> want to be
>>>>>>>>>
>>>>>>>>> able run this
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> in steady state as well as rerun to regenerate
>>>>>>>>> results (or
>>>>>>>>>
>>>>>>>>> catch up if it
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> crashes).
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> There are a couple of tricky things that seem to
>>>>>>>>> make this
>>>>>>>>>
>>>>>>>>> hard with
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> either
>>>>>>>>>     >
>>>>>>>>>     >     >>> of the options proposed:
>>>>>>>>>     >
>>>>>>>>>     >     >>> 1. If I emit this data using event time I have the
>>>>>>>>> problem
>>>>>>>>>
>>>>>>>>> described
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> where
>>>>>>>>>     >
>>>>>>>>>     >     >>> a geographical region with no new clicks or
>>>>>>>>> impressions
>>>>>>>>>
>>>>>>>>> will fail to
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> output
>>>>>>>>>     >
>>>>>>>>>     >     >>> results.
>>>>>>>>>     >
>>>>>>>>>     >     >>> 2. If I emit this data using system time I have the
>>>>>>>>> problem
>>>>>>>>>
>>>>>>>>> that when
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> reprocessing data my window may not be ten minutes
>>>>>>>>> but 10
>>>>>>>>>
>>>>>>>>> hours if my
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> processing is very fast so it dramatically changes
>>>>>>>>> the
>>>>>>>>>
>>>>>>>>> output.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> Maybe a hybrid solution works: I window by event
>>>>>>>>> time but
>>>>>>>>>
>>>>>>>>> trigger results
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> by system time for windows that have updated? Not
>>>>>>>>> really
>>>>>>>>>
>>>>>>>>> sure the details
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> of making that work. Does that work? Are there
>>>>>>>>> concrete
>>>>>>>>>
>>>>>>>>> examples where
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> you
>>>>>>>>>     >
>>>>>>>>>     >     >>> actually want the current behavior?
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> -Jay
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>>>>>>
>>>>>>>>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> wrote:
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> Hi All,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> Thanks for the KIP. We were also in need of a
>>>>>>>>> mechanism to
>>>>>>>>>
>>>>>>>>> trigger
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> punctuate in the absence of events.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> As I described in [
>>>>>>>>>     >
>>>>>>>>>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>>>>>     >
>>>>>>>>>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>>>>>     >
>>>>>>>>>     >     >>>> plugin.system.issuetabpanels:comment-
>>>>>>>>> tabpanel#comment-
>>>>>>>>>
>>>>>>>>> 15926036
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> ],
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - Our approached involved using the event time
>>>>>>>>> by
>>>>>>>>>
>>>>>>>>> default.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - The method to check if there is any punctuate
>>>>>>>>> ready
>>>>>>>>>
>>>>>>>>> in the
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    PunctuationQueue is triggered via the any event
>>>>>>>>>
>>>>>>>>> received by the
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> stream
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    tread, or at the polling intervals in the
>>>>>>>>> absence of
>>>>>>>>>
>>>>>>>>> any events.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - When we create Punctuate objects (which
>>>>>>>>> contains the
>>>>>>>>>
>>>>>>>>> next event
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> time
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    for punctuation and interval), we also record
>>>>>>>>> the
>>>>>>>>>
>>>>>>>>> creation time
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> (system
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    time).
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - While checking for maturity of Punctuate
>>>>>>>>> Schedule by
>>>>>>>>>     >
>>>>>>>>>     >     >> mayBePunctuate
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    method, we also check if the system clock has
>>>>>>>>> elapsed
>>>>>>>>>
>>>>>>>>> the punctuate
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    interval since the schedule creation time.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - In the absence of any event, or in the
>>>>>>>>> absence of any
>>>>>>>>>
>>>>>>>>> event for
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> one
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    topic in the partition group assigned to the
>>>>>>>>> stream
>>>>>>>>>
>>>>>>>>> task, the system
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> time
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    will elapse the interval and we trigger a
>>>>>>>>> punctuate
>>>>>>>>>
>>>>>>>>> using the
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> expected
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    punctuation event time.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - we then create the next punctuation schedule
>>>>>>>>> as
>>>>>>>>>
>>>>>>>>> punctuation event
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> time
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    + punctuation interval, [again recording the
>>>>>>>>> system
>>>>>>>>>
>>>>>>>>> time of creation
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> of
>>>>>>>>>     >
>>>>>>>>>     >     >>>> the
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    schedule].
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>>>>>> approach
>>>>>>>>>
>>>>>>>>> has pros and
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> cons.
>>>>>>>>>     >
>>>>>>>>>     >     >>>> Pros
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - Punctuates will happen in <punctuate
>>>>>>>>> interval> time
>>>>>>>>>
>>>>>>>>> duration at
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> max
>>>>>>>>>     >
>>>>>>>>>     >     >>> in
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    terms of system time.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - The semantics as a whole continues to revolve
>>>>>>>>> around
>>>>>>>>>
>>>>>>>>> event time.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - We can use the old data [old timestamps] to
>>>>>>>>> rerun any
>>>>>>>>>
>>>>>>>>> experiments
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> or
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    tests.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> Cons
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - In case the  <punctuate interval> is not a
>>>>>>>>> time
>>>>>>>>>
>>>>>>>>> duration [say
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> logical
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    time/event count], then the approach might not
>>>>>>>>> be
>>>>>>>>>
>>>>>>>>> meaningful.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - In case there is a case where we have to wait
>>>>>>>>> for an
>>>>>>>>>
>>>>>>>>> actual event
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> from
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    a low event rate partition in the partition
>>>>>>>>> group, this
>>>>>>>>>
>>>>>>>>> approach
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> will
>>>>>>>>>     >
>>>>>>>>>     >     >>>> jump
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    the gun.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    - in case the event processing cannot catch up
>>>>>>>>> with the
>>>>>>>>>
>>>>>>>>> event rate
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> and
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    the expected timestamp events gets queued for
>>>>>>>>> long
>>>>>>>>>
>>>>>>>>> time, this
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> approach
>>>>>>>>>     >
>>>>>>>>>     >     >>>>    might jump the gun.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> I believe the above approach and discussion goes
>>>>>>>>> close to
>>>>>>>>>
>>>>>>>>> the approach
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> A.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> -----------
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> I like the idea of having an even count based
>>>>>>>>> punctuate.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> -----------
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> I agree with the discussion around approach C,
>>>>>>>>> that we
>>>>>>>>>
>>>>>>>>> should provide
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> the
>>>>>>>>>     >
>>>>>>>>>     >     >>>> user with the option to choose system time or
>>>>>>>>> event time
>>>>>>>>>
>>>>>>>>> based
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> punctuates.
>>>>>>>>>     >
>>>>>>>>>     >     >>>> But I believe that the user predominantly wants to
>>>>>>>>> use
>>>>>>>>>
>>>>>>>>> event time while
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> not
>>>>>>>>>     >
>>>>>>>>>     >     >>>> missing out on regular punctuates due to event
>>>>>>>>> delays or
>>>>>>>>>
>>>>>>>>> event
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> absences.
>>>>>>>>>     >
>>>>>>>>>     >     >>>> Hence a complex punctuate option as Matthias
>>>>>>>>> mentioned
>>>>>>>>>
>>>>>>>>> (quoted below)
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> would
>>>>>>>>>     >
>>>>>>>>>     >     >>>> be most apt.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> "- We might want to add "complex" schedules later
>>>>>>>>> on
>>>>>>>>>
>>>>>>>>> (like, punctuate
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> on
>>>>>>>>>     >
>>>>>>>>>     >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>>>>>> time
>>>>>>>>>
>>>>>>>>> whatever comes
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> first)."
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> -----------
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> I think I read somewhere that Kafka Streams
>>>>>>>>> started with
>>>>>>>>>
>>>>>>>>> System Time as
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> the
>>>>>>>>>     >
>>>>>>>>>     >     >>>> punctuation standard, but was later changed to
>>>>>>>>> Event Time.
>>>>>>>>>
>>>>>>>>> I guess
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> there
>>>>>>>>>     >
>>>>>>>>>     >     >>>> would be some good reason behind it. As Kafka
>>>>>>>>> Streams want
>>>>>>>>>
>>>>>>>>> to evolve
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> more
>>>>>>>>>     >
>>>>>>>>>     >     >>>> on the Stream Processing front, I believe the
>>>>>>>>> emphasis on
>>>>>>>>>
>>>>>>>>> event time
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> would
>>>>>>>>>     >
>>>>>>>>>     >     >>>> remain quite strong.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> With Regards,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> Arun Mathew
>>>>>>>>>     >
>>>>>>>>>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>>>>>>
>>>>>>>>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> wrote:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> Yeah I like PuncutationType much better; I just
>>>>>>>>> threw
>>>>>>>>>
>>>>>>>>> Time out there
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>>>>>> still
>>>>>>>>>
>>>>>>>>> think it's
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> worth considering what this buys us over an
>>>>>>>>> additional
>>>>>>>>>
>>>>>>>>> callback. I
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> foresee a number of punctuate implementations
>>>>>>>>> following
>>>>>>>>>
>>>>>>>>> this pattern:
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> public void punctuate(PunctuationType type) {
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>     switch (type) {
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>         case EVENT_TIME:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>             methodA();
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>             break;
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>         case SYSTEM_TIME:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>             methodB();
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>             break;
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>     }
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> }
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> I guess one advantage of this approach is we
>>>>>>>>> could add
>>>>>>>>>
>>>>>>>>> additional
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> punctuation types later in a backwards compatible
>>>>>>>>> way
>>>>>>>>>
>>>>>>>>> (like event
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> count
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> as you mentioned).
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> -Tommy
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>>>>>> Sax wrote:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> That sounds promising.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> I am just wondering if `Time` is the best name.
>>>>>>>>> Maybe we
>>>>>>>>>
>>>>>>>>> want to
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> add
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> other non-time based punctuations at some point
>>>>>>>>> later. I
>>>>>>>>>
>>>>>>>>> would
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> suggest
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> enum PunctuationType {
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>   EVENT_TIME,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>   SYSTEM_TIME,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> }
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> or similar. Just to keep the door open -- it's
>>>>>>>>> easier to
>>>>>>>>>
>>>>>>>>> add new
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> stuff
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> if the name is more generic.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> -Matthias
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> I agree that the framework providing and
>>>>>>>>> managing the
>>>>>>>>>
>>>>>>>>> notion of
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> stream
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> time is valuable and not something we would
>>>>>>>>> want to
>>>>>>>>>
>>>>>>>>> delegate to
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> the
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>>>>>> separate
>>>>>>>>>
>>>>>>>>> callback
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> (option
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> C)
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> is that messy (it could just be a default
>>>>>>>>> method with
>>>>>>>>>
>>>>>>>>> an empty
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> implementation), but if we wanted a single API
>>>>>>>>> to
>>>>>>>>>
>>>>>>>>> handle both
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> cases,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> how about something like the following?
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> enum Time {
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>    STREAM,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>    CLOCK
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> }
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> Then on ProcessorContext:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>>>>>> We could
>>>>>>>>>
>>>>>>>>> allow
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> this
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> to
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> be called once for each value of time to mix
>>>>>>>>>
>>>>>>>>> approaches.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> Then the Processor API becomes:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>>>>>>
>>>>>>>>> schedule resulted
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> in
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> this call.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> Thoughts?
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>>>>>> Sax
>>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> I was thinking about the four options you
>>>>>>>>> proposed in
>>>>>>>>>
>>>>>>>>> more
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> details
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> and
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> this are my thoughts:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> (A) You argue, that users can still
>>>>>>>>> "punctuate" on
>>>>>>>>>
>>>>>>>>> event-time
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> via
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> process(), but I am not sure if this is
>>>>>>>>> possible.
>>>>>>>>>
>>>>>>>>> Note, that
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> users
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> only
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> get record timestamps via context.timestamp().
>>>>>>>>> Thus,
>>>>>>>>>
>>>>>>>>> users
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> would
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> need
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> to
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> track the time progress per partition (based
>>>>>>>>> on the
>>>>>>>>>
>>>>>>>>> partitions
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> they
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> obverse via context.partition(). (This alone
>>>>>>>>> puts a
>>>>>>>>>
>>>>>>>>> huge burden
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> on
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> the
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> user by itself.) However, users are not
>>>>>>>>> notified at
>>>>>>>>>
>>>>>>>>> startup
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> what
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> partitions are assigned, and user are not
>>>>>>>>> notified
>>>>>>>>>
>>>>>>>>> when
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> partitions
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> get
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> revoked. Because this information is not
>>>>>>>>> available,
>>>>>>>>>
>>>>>>>>> it's not
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> possible
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> to
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> "manually advance" stream-time, and thus
>>>>>>>>> event-time
>>>>>>>>>
>>>>>>>>> punctuation
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> within
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> process() seems not to be possible -- or do
>>>>>>>>> you see a
>>>>>>>>>
>>>>>>>>> way to
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> get
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> it
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> done? And even if, it might still be too
>>>>>>>>> clumsy to
>>>>>>>>>
>>>>>>>>> use.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> (B) This does not allow to mix both
>>>>>>>>> approaches, thus
>>>>>>>>>
>>>>>>>>> limiting
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> what
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> users
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> can do.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> (C) This should give all flexibility we need.
>>>>>>>>> However,
>>>>>>>>>
>>>>>>>>> just
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> adding
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> one
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> more method seems to be a solution that is too
>>>>>>>>> simple
>>>>>>>>>
>>>>>>>>> (cf my
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> comments
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> below).
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>>>>>> sure how
>>>>>>>>>
>>>>>>>>> a user
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> could
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> enable system-time and event-time punctuation
>>>>>>>>> in
>>>>>>>>>
>>>>>>>>> parallel.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> Overall options (C) seems to be the most
>>>>>>>>> promising
>>>>>>>>>
>>>>>>>>> approach to
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> me.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> Because I also favor a clean API, we might
>>>>>>>>> keep
>>>>>>>>>
>>>>>>>>> current
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> punctuate()
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>>>>>> at some
>>>>>>>>>
>>>>>>>>> later
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> point
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> when
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> people use the "new punctuate API".
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> Couple of follow up questions:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> - I am wondering, if we should have two
>>>>>>>>> callback
>>>>>>>>>
>>>>>>>>> methods or
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> just
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> one
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> (ie, a unified for system and event time
>>>>>>>>> punctuation
>>>>>>>>>
>>>>>>>>> or one for
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> each?).
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> - If we have one, how can the user figure out,
>>>>>>>>> which
>>>>>>>>>
>>>>>>>>> condition
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> did
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> trigger?
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> - How would the API look like, for registering
>>>>>>>>>
>>>>>>>>> different
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> punctuate
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> - We might want to add "complex" schedules
>>>>>>>>> later on
>>>>>>>>>
>>>>>>>>> (like,
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> punctuate
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> on
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>>>>>> system-time
>>>>>>>>>
>>>>>>>>> whatever
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> comes
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> first). I don't say we should add this right
>>>>>>>>> away, but
>>>>>>>>>
>>>>>>>>> we might
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> want
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> to
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> define the API in a way, that it allows
>>>>>>>>> extensions
>>>>>>>>>
>>>>>>>>> like this
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> later
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> on,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> without redesigning the API (ie, the API
>>>>>>>>> should be
>>>>>>>>>
>>>>>>>>> designed
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> extensible)
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> - Did you ever consider count-based
>>>>>>>>> punctuation?
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> I understand, that you would like to solve a
>>>>>>>>> simple
>>>>>>>>>
>>>>>>>>> problem,
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> but
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> we
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> learned from the past, that just "adding some
>>>>>>>>> API"
>>>>>>>>>
>>>>>>>>> quickly
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> leads
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> to a
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> not very well defined API that needs time
>>>>>>>>> consuming
>>>>>>>>>
>>>>>>>>> clean up
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> later on
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>>>>>> holistic
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> punctuation
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> KIP
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> with this from the beginning on to avoid later
>>>>>>>>> painful
>>>>>>>>>     >
>>>>>>>>>     >     >> redesign.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> -Matthias
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> Thanks Thomas,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> I'm also wary of changing the existing
>>>>>>>>> semantics of
>>>>>>>>>     >
>>>>>>>>>     >     >> punctuate,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> for
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> backward compatibility reasons, although I
>>>>>>>>> like the
>>>>>>>>>     >
>>>>>>>>>     >     >> conceptual
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> simplicity of that option.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>>>>>> a way,
>>>>>>>>>
>>>>>>>>> uglier.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> I
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> added
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> this to the KIP now as option (C).
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>>>>>> more
>>>>>>>>>
>>>>>>>>> flexible,
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> as
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> it
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> allows
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> you to return any value, you're not limited
>>>>>>>>> to event
>>>>>>>>>
>>>>>>>>> time or
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> system
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> time
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> (although I don't see an actual use case
>>>>>>>>> where you
>>>>>>>>>
>>>>>>>>> might need
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> anything
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> else then those two). Hence I also proposed
>>>>>>>>> the
>>>>>>>>>
>>>>>>>>> option to
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> allow
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> users
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>>>>>> for
>>>>>>>>>
>>>>>>>>> them given
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> the
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> presence or absence of messages, much like
>>>>>>>>> they can
>>>>>>>>>
>>>>>>>>> decide
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> what
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> msg
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> time
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>>>>>> What do
>>>>>>>>>
>>>>>>>>> you
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> think
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> about
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> that? This is probably most flexible but also
>>>>>>>>> most
>>>>>>>>>     >
>>>>>>>>>     >     >> complicated.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> All comments appreciated.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> Cheers,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> Michal
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> Although I fully agree we need a way to
>>>>>>>>> trigger
>>>>>>>>>
>>>>>>>>> periodic
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> processing
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> that is independent from whether and when
>>>>>>>>> messages
>>>>>>>>>
>>>>>>>>> arrive,
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> I'm
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> not sure
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> I like the idea of changing the existing
>>>>>>>>> semantics
>>>>>>>>>
>>>>>>>>> across
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> the
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> board.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> What if we added an additional callback to
>>>>>>>>> Processor
>>>>>>>>>
>>>>>>>>> that
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> can
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> be
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>>>>>> always
>>>>>>>>>
>>>>>>>>> called at
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> fixed, wall
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>>>>>> have to
>>>>>>>>>
>>>>>>>>> give
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> up
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> the
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> notion
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> of stream time to be able to do periodic
>>>>>>>>> processing.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>>>>>> Borowiecki
>>>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> Hi all,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>>>>>>
>>>>>>>>> punctuate
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> semantics
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>>>>>>
>>>>>>>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/ confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>>>>>>
>>>>>>>>> 138%
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> 3A+C
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> hange+
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> punctuate+semantics>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> .
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> Appreciating there can be different views
>>>>>>>>> on
>>>>>>>>>
>>>>>>>>> system-time
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> vs
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> event-
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> time
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>>>>>> case and
>>>>>>>>>
>>>>>>>>> the
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> importance of
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>>>>>> I've
>>>>>>>>>
>>>>>>>>> left it
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> quite
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> open
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> and
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>>>>>>
>>>>>>>>> progresses.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> Thanks,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>>>>> Michal
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> --
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>     Tommy Becker
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>     Senior Software Engineer
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>     tivo.com
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> ________________________________
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> This email and any attachments may contain
>>>>>>>>> confidential
>>>>>>>>>
>>>>>>>>> and
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> privileged material for the sole use of the
>>>>>>>>> intended
>>>>>>>>>
>>>>>>>>> recipient.
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> Any
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> review, copying, or distribution of this email
>>>>>>>>> (or any
>>>>>>>>>     >
>>>>>>>>>     >     >> attachments)
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> by others is prohibited. If you are not the
>>>>>>>>> intended
>>>>>>>>>
>>>>>>>>> recipient,
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> please contact the sender immediately and
>>>>>>>>> permanently
>>>>>>>>>
>>>>>>>>> delete this
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> email and any attachments. No employee or agent
>>>>>>>>> of TiVo
>>>>>>>>>
>>>>>>>>> Inc. is
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> authorized to conclude any binding agreement on
>>>>>>>>> behalf
>>>>>>>>>
>>>>>>>>> of TiVo
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> Inc.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>>>>>> only be
>>>>>>>>>
>>>>>>>>> made by a
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>> signed written agreement.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> --
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>     Tommy Becker
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>     Senior Software Engineer
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>     tivo.com
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> ________________________________
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> This email and any attachments may contain
>>>>>>>>> confidential
>>>>>>>>>
>>>>>>>>> and
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> privileged
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> material for the sole use of the intended
>>>>>>>>> recipient. Any
>>>>>>>>>
>>>>>>>>> review,
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>> copying,
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> or distribution of this email (or any
>>>>>>>>> attachments) by
>>>>>>>>>
>>>>>>>>> others is
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>> prohibited.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> If you are not the intended recipient, please
>>>>>>>>> contact the
>>>>>>>>>
>>>>>>>>> sender
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> immediately and permanently delete this email and
>>>>>>>>> any
>>>>>>>>>
>>>>>>>>> attachments. No
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>>>>>> conclude
>>>>>>>>>
>>>>>>>>> any binding
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>>>>>> Binding
>>>>>>>>>
>>>>>>>>> agreements with
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >> TiVo
>>>>>>>>>     >
>>>>>>>>>     >     >>>>> Inc. may only be made by a signed written
>>>>>>>>> agreement.
>>>>>>>>>     >
>>>>>>>>>     >     >>>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>>
>>>>>>>>>     >
>>>>>>>>>     >     >>>
>>>>>>>>>     >
>>>>>>>>>     >     >>
>>>>>>>>>     >
>>>>>>>>>     >     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > --
>>>>>>>>>     >
>>>>>>>>>     > <http://www.openbet.com/> <http://www.openbet.com/>
>>>>>>>>>
>>>>>>>>>     >
>>>>>>>>>     > *Michal Borowiecki*
>>>>>>>>>     >
>>>>>>>>>     > *Senior Software Engineer L4*
>>>>>>>>>     >
>>>>>>>>>     > *T: *
>>>>>>>>>     >
>>>>>>>>>     > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>>>>>>     >
>>>>>>>>>     > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > *E: *
>>>>>>>>>     >
>>>>>>>>>     > michal.borowiecki@openbet.com
>>>>>>>>>     >
>>>>>>>>>     > *W: *
>>>>>>>>>     >
>>>>>>>>>     > www.openbet.com
>>>>>>>>>     >
>>>>>>>>>     > *OpenBet Ltd*
>>>>>>>>>     >
>>>>>>>>>     > Chiswick Park Building 9
>>>>>>>>>     >
>>>>>>>>>     > 566 Chiswick High Rd
>>>>>>>>>     >
>>>>>>>>>     > London
>>>>>>>>>     >
>>>>>>>>>     > W4 5XT
>>>>>>>>>     >
>>>>>>>>>     > UK
>>>>>>>>>     >
>>>>>>>>>     > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     > This message is confidential and intended only for the
>>>>>>>>> addressee.
>>>>>>>>>
>>>>>>>>> If you
>>>>>>>>>
>>>>>>>>>     > have received this message in error, please immediately
>>>>>>>>> notify the
>>>>>>>>>     > postmaster@openbet.com and delete it from your system as
>>>>>>>>> well as
>>>>>>>>>
>>>>>>>>> any
>>>>>>>>>
>>>>>>>>>     > copies. The content of e-mails as well as traffic data may
>>>>>>>>> be
>>>>>>>>>
>>>>>>>>> monitored by
>>>>>>>>>
>>>>>>>>>     > OpenBet for employment and security purposes. To protect
>>>>>>>>> the
>>>>>>>>>
>>>>>>>>> environment
>>>>>>>>>
>>>>>>>>>     > please do not print this e-mail unless necessary. OpenBet
>>>>>>>>> Ltd.
>>>>>>>>>
>>>>>>>>> Registered
>>>>>>>>>
>>>>>>>>>     > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>>>>>> London,
>>>>>>>>>
>>>>>>>>> W4 5XT,
>>>>>>>>>
>>>>>>>>>     > United Kingdom. A company registered in England and Wales.
>>>>>>>>>
>>>>>>>>> Registered no.
>>>>>>>>>
>>>>>>>>>     > 3134634. VAT no. GB927523612
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>     >
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>     Tommy Becker
>>>>>>>>>
>>>>>>>>>     Senior Software Engineer
>>>>>>>>>
>>>>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>     tivo.com
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> ________________________________
>>>>>>>>>
>>>>>>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> <http://www.openbet.com/> Michal Borowiecki
>>>>>>>>> Senior Software Engineer L4
>>>>>>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>> -- 
>>>>>> Signature
>>>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>>>> Senior Software Engineer L4
>>>>>> 	T: 	+44 208 742 1600
>>>>>>
>>>>>> 	
>>>>>> 	+44 203 249 8448
>>>>>>
>>>>>> 	
>>>>>> 	 
>>>>>> 	E: 	michal.borowiecki@openbet.com
>>>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>>>
>>>>>> 	
>>>>>> 	OpenBet Ltd
>>>>>>
>>>>>> 	Chiswick Park Building 9
>>>>>>
>>>>>> 	566 Chiswick High Rd
>>>>>>
>>>>>> 	London
>>>>>>
>>>>>> 	W4 5XT
>>>>>>
>>>>>> 	UK
>>>>>>
>>>>>> 	
>>>>>> <https://www.openbet.com/email_promo>
>>>>>>
>>>>>> This message is confidential and intended only for the addressee. If
>>>>>> you have received this message in error, please immediately notify the
>>>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>>>> from your system as well as any copies. The content of e-mails as well
>>>>>> as traffic data may be monitored by OpenBet for employment and
>>>>>> security purposes. To protect the environment please do not print this
>>>>>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>>>>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>>>>> company registered in England and Wales. Registered no. 3134634. VAT
>>>>>> no. GB927523612
>>>>>>
>>>>> -- 
>>>>> Signature
>>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>>> Senior Software Engineer L4
>>>>> 	T: 	+44 208 742 1600
>>>>>
>>>>> 	
>>>>> 	+44 203 249 8448
>>>>>
>>>>> 	
>>>>> 	 
>>>>> 	E: 	michal.borowiecki@openbet.com
>>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>>
>>>>> 	
>>>>> 	OpenBet Ltd
>>>>>
>>>>> 	Chiswick Park Building 9
>>>>>
>>>>> 	566 Chiswick High Rd
>>>>>
>>>>> 	London
>>>>>
>>>>> 	W4 5XT
>>>>>
>>>>> 	UK
>>>>>
>>>>> 	
>>>>> <https://www.openbet.com/email_promo>
>>>>>
>>>>> This message is confidential and intended only for the addressee. If you
>>>>> have received this message in error, please immediately notify the
>>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>>> from your system as well as any copies. The content of e-mails as well
>>>>> as traffic data may be monitored by OpenBet for employment and security
>>>>> purposes. To protect the environment please do not print this e-mail
>>>>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>>>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>>>> registered in England and Wales. Registered no. 3134634. VAT no.
>>>>> GB927523612
>>>>>
>>> -- 
>>> Signature
>>> <http://www.openbet.com/> 	Michal Borowiecki
>>> Senior Software Engineer L4
>>> 	T: 	+44 208 742 1600
>>>
>>> 	
>>> 	+44 203 249 8448
>>>
>>> 	
>>> 	 
>>> 	E: 	michal.borowiecki@openbet.com
>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>
>>> 	
>>> 	OpenBet Ltd
>>>
>>> 	Chiswick Park Building 9
>>>
>>> 	566 Chiswick High Rd
>>>
>>> 	London
>>>
>>> 	W4 5XT
>>>
>>> 	UK
>>>
>>> 	
>>> <https://www.openbet.com/email_promo>
>>>
>>> This message is confidential and intended only for the addressee. If you
>>> have received this message in error, please immediately notify the
>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>> from your system as well as any copies. The content of e-mails as well
>>> as traffic data may be monitored by OpenBet for employment and security
>>> purposes. To protect the environment please do not print this e-mail
>>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>> registered in England and Wales. Registered no. 3134634. VAT no.
>>> GB927523612
>>>
> 
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
> 
> 	
> 	+44 203 249 8448
> 
> 	
> 	 
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
> 
> 	
> 	OpenBet Ltd
> 
> 	Chiswick Park Building 9
> 
> 	566 Chiswick High Rd
> 
> 	London
> 
> 	W4 5XT
> 
> 	UK
> 
> 	
> <https://www.openbet.com/email_promo>
> 
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com <ma...@openbet.com> and delete it
> from your system as well as any copies. The content of e-mails as well
> as traffic data may be monitored by OpenBet for employment and security
> purposes. To protect the environment please do not print this e-mail
> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> registered in England and Wales. Registered no. 3134634. VAT no.
> GB927523612
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I agree with this. We would need to allow processor level configuration.

And I also agree, that the global caching config is not optimal...


-Matthias

On 4/24/17 3:55 AM, Michal Borowiecki wrote:
> Further to this, on your point about configuration:
> 
>> Thus, I also believe that one might need different "configuration"
>> values for the hybrid approach if you run the same code for different
>> scenarios: regular processing, re-processing, catching up scenario. And
>> as the term "configuration" implies, we might be better off to not mix
>> configuration with business logic that is expressed via code.
> I'm not sure I understand what you are suggesting here.
> 
> Configuration is global to a KafkaStreams instance and users might want
> to have different tolerance in different parts of the topology. They
> shouldn't be locked into one value set via global config.
> 
> To illustrate this point: Lately I have discovered the cache config
> introduced in KIP-63
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-63%3A+Unify+store+and+downstream+caching+in+streams>
> and found it quite annoying that it's controlled by a config item. IMO,
> I should be able to control flushing per processor, not be forced to use
> one global value defined in configs.
> 
> It's easy enough for users to source a user-defined config and provided
> it as a parameter to a /given /processor as needed.
> 
> In principal I agree that configuration and business logic are better
> not mixed together but then the configuration mechanism should allow
> users to target specific processors and not be global to the
> KafkaStreams instance.
> 
> Thanks,
> 
> Michal
> 
> On 24/04/17 10:23, Michal Borowiecki wrote:
>>
>> Hi Matthias,
>>
>> I agree it's difficult to reason about the hybrid approach, I
>> certainly found it hard and I'm totally on board with the mantra.
>>
>> I'd be happy to limit the scope of this KIP to add system-time
>> punctuation semantics (in addition to existing stream-time semantics)
>> and leave more complex schemes for users to implement on top of that.
>>
>> Further additional PunctuationTypes, could then be added by future
>> KIPs, possibly including the hybrid approach once it has been given
>> more thought.
>>
>>> There are real-time applications, that want to get
>>> callbacks in regular system-time intervals (completely independent from
>>> stream-time).
>> Can you please describe what they are, so that I can put them on the
>> wiki for later reference?
>>
>> Thanks,
>>
>> Michal
>>
>>
>> On 23/04/17 21:27, Matthias J. Sax wrote:
>>> Hi,
>>>
>>> I do like Damian's API proposal about the punctuation callback function.
>>>
>>> I also did reread the KIP and thought about the semantics we want to
>>> provide.
>>>
>>>> Given the above, I don't see a reason any more for a separate system-time based punctuation.
>>> I disagree here. There are real-time applications, that want to get
>>> callbacks in regular system-time intervals (completely independent from
>>> stream-time). Thus we should allow this -- if we really follow the
>>> "hybrid" approach, this could be configured with stream-time interval
>>> infinite and delay whatever system-time punctuation interval you want to
>>> have. However, I would like to add a proper API for this and do this
>>> configuration under the hood (that would allow one implementation within
>>> all kind of branching for different cases).
>>>
>>> Thus, we definitely should have PunctutionType#StreamTime and
>>> #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
>>> a fan of your latest API proposal.
>>>
>>>
>>> About the hybrid approach in general. On the one hand I like it, on the
>>> other hand, it seems to be rather (1) complicated (not necessarily from
>>> an implementation point of view, but for people to understand it) and
>>> (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
>>>
>>>> It may appear complicated at first but I do think these semantics will
>>>> still be more understandable to users than having 2 separate punctuation
>>>> schedules/callbacks with different PunctuationTypes.
>>> This statement only holds if you apply strong assumptions that I don't
>>> believe hold in general -- see (2) for details -- and I think it is
>>> harder than you assume to reason about the hybrid approach in general.
>>> IMHO, the hybrid approach is a "false friend" that seems to be easy to
>>> reason about...
>>>
>>>
>>> (1) Streams always embraced "easy to use" and we should really be
>>> careful to keep it this way. On the other hand, as we are talking about
>>> changes to PAPI, it won't affect DSL users (DSL does not use punctuation
>>> at all at the moment), and thus, the "easy to use" mantra might not be
>>> affected, while it will allow advanced users to express more complex stuff.
>>>
>>> I like the mantra: "make simple thing easy and complex things possible".
>>>
>>> (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
>>> implicit assumption that even-time progresses at the same "speed" as
>>> system-time during regular processing. This implies the assumption that
>>> a slower progress in stream-time indicates the absence of input events
>>> (and that later arriving input events will have a larger event-time with
>>> high probability). Even if this might be true for some use cases, I
>>> doubt it holds in general. Assume that you get a spike in traffic and
>>> for some reason stream-time does advance slowly because you have more
>>> records to process. This might trigger a system-time based punctuation
>>> call even if this seems not to be intended. I strongly believe that it
>>> is not easy to reason about the semantics of the hybrid approach (even
>>> if the intentional semantics would be super useful -- but I doubt that
>>> we get want we ask for).
>>>
>>> Thus, I also believe that one might need different "configuration"
>>> values for the hybrid approach if you run the same code for different
>>> scenarios: regular processing, re-processing, catching up scenario. And
>>> as the term "configuration" implies, we might be better off to not mix
>>> configuration with business logic that is expressed via code.
>>>
>>>
>>> One more comment: I also don't think that the hybrid approach is
>>> deterministic as claimed in the use-case subpage. I understand the
>>> reasoning and agree, that it is deterministic if certain assumptions
>>> hold -- compare above -- and if configured correctly. But strictly
>>> speaking it's not because there is a dependency on system-time (and
>>> IMHO, if system-time is involved it cannot be deterministic by definition).
>>>
>>>
>>>> I see how in theory this could be implemented on top of the 2 punctuate
>>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>>> the other system-time based) but it would be a much more complicated
>>>> scheme and I don't want to suggest that.
>>> I agree that expressing the intended hybrid semantics is harder if we
>>> offer only #StreamTime and #SystemTime punctuation. However, I also
>>> believe that the hybrid approach is a "false friend" with regard to
>>> reasoning about the semantics (it indicates that it more easy as it is
>>> in reality). Therefore, we might be better off to not offer the hybrid
>>> approach and make it clear to a developed, that it is hard to mix
>>> #StreamTime and #SystemTime in a semantically sound way.
>>>
>>>
>>> Looking forward to your feedback. :)
>>>
>>> -Matthias
>>>
>>>
>>>
>>>
>>> On 4/22/17 11:43 AM, Michal Borowiecki wrote:
>>>> Hi all,
>>>>
>>>> Looking for feedback on the functional interface approach Damian
>>>> proposed. What do people think?
>>>>
>>>> Further on the semantics of triggering punctuate though:
>>>>
>>>> I ran through the 2 use cases that Arun had kindly put on the wiki
>>>> (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
>>>> in my head and on a whiteboard and I can't find a better solution than
>>>> the "hybrid" approach he had proposed.
>>>>
>>>> I see how in theory this could be implemented on top of the 2 punctuate
>>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>>> the other system-time based) but it would be a much more complicated
>>>> scheme and I don't want to suggest that.
>>>>
>>>> However, to add to the hybrid algorithm proposed, I suggest one
>>>> parameter to that: a tolerance period, expressed in milliseconds
>>>> system-time, after which the punctuation will be invoked in case the
>>>> stream-time advance hasn't triggered it within the requested interval
>>>> since the last invocation of punctuate (as recorded in system-time)
>>>>
>>>> This would allow a user-defined tolerance for late arriving events. The
>>>> trade off would be left for the user to decide: regular punctuation in
>>>> the case of absence of events vs allowing for records arriving late or
>>>> some build-up due to processing not catching up with the event rate.
>>>> In the one extreme, this tolerance could be set to infinity, turning
>>>> hybrid into simply stream-time based punctuate, like we have now. In the
>>>> other extreme, the tolerance could be set to 0, resulting in a
>>>> system-time upper bound on the effective punctuation interval.
>>>>
>>>> Given the above, I don't see a reason any more for a separate
>>>> system-time based punctuation. The "hybrid" approach with 0ms tolerance
>>>> would under normal operation trigger at regular intervals wrt the
>>>> system-time, except in cases of re-play/catch-up, where the stream time
>>>> advances faster than system time. In these cases punctuate would happen
>>>> more often than the specified interval wrt system time. However, the
>>>> use-cases that need system-time punctuations (that I've seen at least)
>>>> really only have a need for an upper bound on punctuation delay but
>>>> don't need a lower bound.
>>>>
>>>> To that effect I'd propose the api to be as follows, on ProcessorContext:
>>>>
>>>> schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
>>>>
>>>> schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
>>>>
>>>> Punctuation is triggered when either:
>>>> - the stream time advances past the (stream time of the previous
>>>> punctuation) + interval;
>>>> - or (iff the toleranceInterval is set) when the system time advances
>>>> past the (system time of the previous punctuation) + interval +
>>>> toleranceInterval
>>>>
>>>> In either case:
>>>> - we trigger punctuate passing as the argument the stream time at which
>>>> the current punctuation was meant to happen
>>>> - next punctuate is scheduled at (stream time at which the current
>>>> punctuation was meant to happen) + interval
>>>>
>>>> It may appear complicated at first but I do think these semantics will
>>>> still be more understandable to users than having 2 separate punctuation
>>>> schedules/callbacks with different PunctuationTypes.
>>>>
>>>>
>>>>
>>>> PS. Having re-read this, maybe the following alternative would be easier
>>>> to understand (WDYT?):
>>>>
>>>> schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
>>>>
>>>> schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
>>>>
>>>> Punctuation is triggered when either:
>>>> - the stream time advances past the (stream time of the previous
>>>> punctuation) + streamTimeInterval;
>>>> - or (iff systemTimeUpperBound is set) when the system time advances
>>>> past the (system time of the previous punctuation) + systemTimeUpperBound
>>>>
>>>> Awaiting comments.
>>>>
>>>> Thanks,
>>>> Michal
>>>>
>>>> On 21/04/17 16:56, Michal Borowiecki wrote:
>>>>> Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>>>>> in favour of something that doesn't return a record. Not a problem though.
>>>>>
>>>>>
>>>>> On 21/04/17 16:32, Damian Guy wrote:
>>>>>> Thanks Michal,
>>>>>> I agree Transformer.punctuate should also be void, but we can deprecate
>>>>>> that too in favor of the new interface.
>>>>>>
>>>>>> Thanks for the javadoc PR!
>>>>>>
>>>>>> Cheers,
>>>>>> Damian
>>>>>>
>>>>>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>
>>>>>>> Yes, that looks better to me.
>>>>>>>
>>>>>>> Note that punctuate on Transformer is currently returning a record, but I
>>>>>>> think it's ok to have all output records be sent via
>>>>>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>>>>>> multiple records from one invocation of punctuate.
>>>>>>>
>>>>>>> This way it's consistent between Processor and Transformer.
>>>>>>>
>>>>>>>
>>>>>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>>>>>> there:
>>>>>>>
>>>>>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>>>>>
>>>>>>> and PR: https://github.com/apache/kafka/pull/2884
>>>>>>>
>>>>>>> Cheers,
>>>>>>>
>>>>>>> Michal
>>>>>>> On 20/04/17 18:55, Damian Guy wrote:
>>>>>>>
>>>>>>> Hi Michal,
>>>>>>>
>>>>>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>>>>>> the API.
>>>>>>> 1. deprecate the punctuate method on Processor
>>>>>>> 2. create a new Functional Interface just for Punctuation, something like:
>>>>>>> interface Punctuator {
>>>>>>>     void punctuate(long timestamp)
>>>>>>> }
>>>>>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>>>>>> interval, PunctuationType type, Punctuator callback)
>>>>>>> 4. deprecate the existing schedule function
>>>>>>>
>>>>>>> Thoughts?
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Damian
>>>>>>>
>>>>>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>>
>>>>>>>> Hi Thomas,
>>>>>>>>
>>>>>>>> I would say our use cases fall in the same category as yours.
>>>>>>>>
>>>>>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>>>>>
>>>>>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>>>>>> of design:
>>>>>>>>
>>>>>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>>>>>
>>>>>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>>>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>>>>>
>>>>>>>> Since the output field X is derived in some non-trivial way, we don't
>>>>>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>>>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>>>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>>>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>>>>>> another input field Z).
>>>>>>>>
>>>>>>>> So we have kv stores with the records and an additional kv store with
>>>>>>>> timestamp->id mapping which act like an index where we periodically do a
>>>>>>>> ranged query.
>>>>>>>>
>>>>>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>>>>>> work when there were no regular msgs on the input topic.
>>>>>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>>>>>> outside we created a "ticker" that produced msgs once per second onto
>>>>>>>> another topic and fed it into the same topology to trigger punctuate.
>>>>>>>> This didn't work either, which was much more surprising to us at the
>>>>>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>>>>>> *all* input partitions receive messages regularly.
>>>>>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>>>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>>>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>>>>>> is consumed by the second topology and is its only input topic. There's a
>>>>>>>> transformer on that topic which populates and updates the time-based
>>>>>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>>>>>> elapsed, the record id is sent to the main transformer, which
>>>>>>>> updates/deletes the record from the main kv store and forwards the
>>>>>>>> transformed record to the output topic.
>>>>>>>>
>>>>>>>> To me this setup feels horrendously complicated for what it does.
>>>>>>>>
>>>>>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>>>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>>>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>>>>>> However, the ticks don't feel so hacky when you realise they give you
>>>>>>>> some hypothetical benefits in predictability. You can reprocess the
>>>>>>>> messages in a reproducible manner, since the topologies use event-time,
>>>>>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>>>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>>>>>> reprocessing).
>>>>>>>> To make that work though, we would have to have the stream time advance
>>>>>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>>>>>> presence of messages on the other input topic.
>>>>>>>>
>>>>>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>>>>>> and the hybrid would work to simplify this a lot.
>>>>>>>>
>>>>>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>>>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>>>>>> have to go with punctuate there too.
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Michal
>>>>>>>>
>>>>>>>>
>>>>>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>>>>>
>>>>>>>> Here's an example that we currently have.  We have a streams processor
>>>>>>>> that does a transform from one topic into another. One of the fields in
>>>>>>>> the source topic record is an expiration time, and one of the functions
>>>>>>>> of the processor is to ensure that expired records get deleted promptly
>>>>>>>> after that time passes (typically days or weeks after the message was
>>>>>>>> originally produced). To do that, the processor keeps a state store of
>>>>>>>> keys and expiration times, iterates that store in punctuate(), and
>>>>>>>> emits delete (null) records for expired items. This needs to happen at
>>>>>>>> some minimum interval regardless of the incoming message rate of the
>>>>>>>> source topic.
>>>>>>>>
>>>>>>>> In this scenario, the expiration of records is the primary function of
>>>>>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>>>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>>>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>>>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>>>>>> range of use-cases.
>>>>>>>>
>>>>>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>>>>>
>>>>>>>> I apologize for the longer email below.  To my defense, it started
>>>>>>>> out much
>>>>>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>>>>>> devil's
>>>>>>>> advocate for a number of arguments brought forth in order to help
>>>>>>>> improve
>>>>>>>> this KIP -- I am not implying I necessarily disagree with the
>>>>>>>> arguments.
>>>>>>>>
>>>>>>>> That aside, here are some further thoughts.
>>>>>>>>
>>>>>>>> First, there are (at least?) two categories for actions/behavior you
>>>>>>>> invoke
>>>>>>>> via punctuate():
>>>>>>>>
>>>>>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>>>>>> to
>>>>>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>>>>>> the
>>>>>>>> impact of punctuate is typically not observable by other processing
>>>>>>>> nodes
>>>>>>>> in the topology.
>>>>>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>>>>>> the
>>>>>>>> punctuate is all about being observable by downstream processing
>>>>>>>> nodes.
>>>>>>>>
>>>>>>>> A few releases back, we introduced record caches (DSL) and state
>>>>>>>> store
>>>>>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>>>>>> relating to
>>>>>>>> (2) where some users needed to control -- here: limit -- the
>>>>>>>> downstream
>>>>>>>> output rate of Kafka Streams because the downstream systems/apps
>>>>>>>> would not
>>>>>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>>>>>> their
>>>>>>>> scalability).  The argument for KIP-63, which notably did not
>>>>>>>> introduce a
>>>>>>>> "trigger" API, was that such an interaction with downstream systems
>>>>>>>> is an
>>>>>>>> operational concern;  it should not impact the processing *logic* of
>>>>>>>> your
>>>>>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>>>>>> API,
>>>>>>>> especially not the declarative DSL, with such operational concerns.
>>>>>>>>
>>>>>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>>>>>> sorry, I
>>>>>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>>>>>> observing that our conversation is moving more and more into the
>>>>>>>> direction
>>>>>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>>>>>> for
>>>>>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>>>>>> comments voiced here are about sth like "IF stream-time didn't
>>>>>>>> trigger
>>>>>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>>>>>> want
>>>>>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>>>>>> related
>>>>>>>> note, whatever we are discussing here will impact state store caches
>>>>>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>>>>>> should
>>>>>>>> clarify any such impact here.
>>>>>>>>
>>>>>>>> Switching topics slightly.
>>>>>>>>
>>>>>>>> Jay wrote:
>>>>>>>>
>>>>>>>> One thing I've always found super important for this kind of design
>>>>>>>> work
>>>>>>>> is to do a really good job of cataloging the landscape of use cases
>>>>>>>> and
>>>>>>>> how prevalent each one is.
>>>>>>>>
>>>>>>>> +1 to this, as others have already said.
>>>>>>>>
>>>>>>>> Here, let me highlight -- just in case -- that when we talked about
>>>>>>>> windowing use cases in the recent emails, the Processor API (where
>>>>>>>> `punctuate` resides) does not have any notion of windowing at
>>>>>>>> all.  If you
>>>>>>>> want to do windowing *in the Processor API*, you must do so manually
>>>>>>>> in
>>>>>>>> combination with window stores.  For this reason I'd suggest to
>>>>>>>> discuss use
>>>>>>>> cases not just in general, but also in view of how you'd do so in the
>>>>>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>>>>>> `punctuate`
>>>>>>>> does not impact the DSL at all, unless we add new functionality to
>>>>>>>> it.
>>>>>>>>
>>>>>>>> Jay wrote in his strawman example:
>>>>>>>>
>>>>>>>> You aggregate click and impression data for a reddit like site.
>>>>>>>> Every ten
>>>>>>>> minutes you want to output a ranked list of the top 10 articles
>>>>>>>> ranked by
>>>>>>>> clicks/impressions for each geographical area. I want to be able
>>>>>>>> run this
>>>>>>>> in steady state as well as rerun to regenerate results (or catch up
>>>>>>>> if it
>>>>>>>> crashes).
>>>>>>>>
>>>>>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>>>>>> we
>>>>>>>> argued that the reason for saying "every ten minutes" above is not
>>>>>>>> necessarily about because you want to output data *exactly* after ten
>>>>>>>> minutes, but that you want to perform an aggregation based on 10-
>>>>>>>> minute
>>>>>>>> windows of input data; i.e., the point is about specifying the input
>>>>>>>> for
>>>>>>>> your aggregation, not or less about when the results of the
>>>>>>>> aggregation
>>>>>>>> should be send downstream.  To take an extreme example, you could
>>>>>>>> disable
>>>>>>>> record caches and let your app output a downstream update for every
>>>>>>>> incoming input record.  If the last input record was from at minute 7
>>>>>>>> of 10
>>>>>>>> (for a 10-min window), then what your app would output at minute 10
>>>>>>>> would
>>>>>>>> be identical to what it had already emitted at minute 7 earlier
>>>>>>>> anyways.
>>>>>>>> This is particularly true when we take late-arriving data into
>>>>>>>> account:  if
>>>>>>>> a late record arrived at minute 13, your app would (by default) send
>>>>>>>> a new
>>>>>>>> update downstream, even though the "original" 10 minutes have already
>>>>>>>> passed.
>>>>>>>>
>>>>>>>> Jay wrote...:
>>>>>>>>
>>>>>>>> There are a couple of tricky things that seem to make this hard
>>>>>>>> with
>>>>>>>>
>>>>>>>> either
>>>>>>>>
>>>>>>>> of the options proposed:
>>>>>>>> 1. If I emit this data using event time I have the problem
>>>>>>>> described where
>>>>>>>> a geographical region with no new clicks or impressions will fail
>>>>>>>> to
>>>>>>>>
>>>>>>>> output
>>>>>>>>
>>>>>>>> results.
>>>>>>>>
>>>>>>>> ...and Arun Mathew wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> We window by the event time, but trigger punctuate in <punctuate
>>>>>>>> interval>
>>>>>>>> duration of system time, in the absence of an event crossing the
>>>>>>>> punctuate
>>>>>>>> event time.
>>>>>>>>
>>>>>>>> So, given what I wrote above about the status quo and what you can
>>>>>>>> already
>>>>>>>> do with it, is the concern that the state store cache doesn't give
>>>>>>>> you
>>>>>>>> *direct* control over "forcing an output after no later than X
>>>>>>>> seconds [of
>>>>>>>> processing-time]" but only indirect control through a cache
>>>>>>>> size?  (Note
>>>>>>>> that I am not dismissing the claims why this might be helpful.)
>>>>>>>>
>>>>>>>> Arun Mathew wrote:
>>>>>>>>
>>>>>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>>>>>> output the
>>>>>>>> event counts on each topic on each cluster aggregated over a 1
>>>>>>>> minute
>>>>>>>> window. We have to use event time to be able to cross check the
>>>>>>>> counts.
>>>>>>>>
>>>>>>>> But
>>>>>>>>
>>>>>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>>>>>> time in
>>>>>>>>
>>>>>>>> the
>>>>>>>>
>>>>>>>> absence of events. Otherwise the event counts for unexpired windows
>>>>>>>> would
>>>>>>>> be 0 which is bad.
>>>>>>>>
>>>>>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>>>>>> absence
>>>>>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>>>>>> Scala
>>>>>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>>>>>> root
>>>>>>>> cause that the downstream system interprets the absence of output in
>>>>>>>> a
>>>>>>>> particular way ("No output after 1 minute = I consider the output to
>>>>>>>> be
>>>>>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>>>>>> possible)
>>>>>>>> to correctly handle the difference between absence of output vs.
>>>>>>>> output of
>>>>>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>>>>>> but
>>>>>>>> want to understand the motivation better. :-)
>>>>>>>>
>>>>>>>> Also, to add some perspective, in some related discussions we talked
>>>>>>>> about
>>>>>>>> how a Kafka Streams application should not worry or not be coupled
>>>>>>>> unnecessarily with such interpretation specifics in a downstream
>>>>>>>> system's
>>>>>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>>>>>> more
>>>>>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>>>>>> than
>>>>>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>>>>>> and
>>>>>>>> downstream systems, including helping to reconcile the differences in
>>>>>>>> how
>>>>>>>> these systems interpret changes, updates, late-arriving data,
>>>>>>>> etc.  Kafka
>>>>>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>>>>>> processing from the specifics of downstream systems, thanks to
>>>>>>>> specific
>>>>>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>>>>>> this
>>>>>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>>>>>> but
>>>>>>>> it's currently awkward to use Connect for this", this might be a
>>>>>>>> problem we
>>>>>>>> can solve, too.)
>>>>>>>>
>>>>>>>> Switching topics slightly again.
>>>>>>>>
>>>>>>>> Thomas wrote:
>>>>>>>>
>>>>>>>> I'm not entirely convinced that a separate callback (option C)
>>>>>>>> is that messy (it could just be a default method with an empty
>>>>>>>> implementation), but if we wanted a single API to handle both
>>>>>>>> cases,
>>>>>>>> how about something like the following?
>>>>>>>>
>>>>>>>> enum Time {
>>>>>>>>    STREAM,
>>>>>>>>    CLOCK
>>>>>>>> }
>>>>>>>>
>>>>>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>>>>>> then
>>>>>>>> whatever the user is doing inside this method is a black box to Kafka
>>>>>>>> Streams (similar to how we have no idea what the user does inside a
>>>>>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>>>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>>>>>> won't
>>>>>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>>>>>> frequency than the processing-time action.  Or, we won't know whether
>>>>>>>> the
>>>>>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>>>>>> minute of
>>>>>>>> stream-time or 1-minute of processing-time, whichever comes
>>>>>>>> first").  That
>>>>>>>> said, I am not certain yet whether we would need such knowledge
>>>>>>>> because,
>>>>>>>> when using the Processor API, most of the work and decisions must be
>>>>>>>> done
>>>>>>>> by the user anyways.  It would matter though if the concept of
>>>>>>>> "triggers"
>>>>>>>> were to bubble up into the DSL because in the DSL the management of
>>>>>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>>>>>> Streams.
>>>>>>>>
>>>>>>>> [In any case, btw, we have the corner case where the user configured
>>>>>>>> the
>>>>>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>>>>>> extractor), at which point both punctuate variants are based on the
>>>>>>>> same
>>>>>>>> time semantics / timeline.]
>>>>>>>>
>>>>>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>>>>>> it
>>>>>>>> this far. :-)
>>>>>>>>
>>>>>>>> More than happy to hear your thoughts!
>>>>>>>> Michael
>>>>>>>>
>>>>>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> Thanks Matthias.
>>>>>>>> Sure, will correct it right away.
>>>>>>>>
>>>>>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> Thanks for preparing this page!
>>>>>>>>
>>>>>>>> About terminology:
>>>>>>>>
>>>>>>>> You introduce the term "event time" -- but we should call this
>>>>>>>> "stream
>>>>>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>>>>>> this
>>>>>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>>>>>
>>>>>>>> Does this make sense to you?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>>>>
>>>>>>>> Thanks Ewen.
>>>>>>>>
>>>>>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>>>>>
>>>>>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>>>>
>>>>>>>>
>>>>>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>>>>>> public.
>>>>>>>>
>>>>>>>> --
>>>>>>>> Arun Mathew
>>>>>>>>
>>>>>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>     Arun,
>>>>>>>>
>>>>>>>>     I've given you permission to edit the wiki. Let me know if
>>>>>>>> you run
>>>>>>>>
>>>>>>>> into any
>>>>>>>>
>>>>>>>>     issues.
>>>>>>>>
>>>>>>>>     -Ewen
>>>>>>>>
>>>>>>>>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp> <am...@yahoo-corp.jp>
>>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>>     > Thanks Michal. I don’t have the access yet [arunmathew88].
>>>>>>>> Should I
>>>>>>>>
>>>>>>>> be
>>>>>>>>
>>>>>>>>     > sending a separate mail for this?
>>>>>>>>     >
>>>>>>>>     > I thought one of the person following this thread would be
>>>>>>>> able to
>>>>>>>>
>>>>>>>> give me
>>>>>>>>
>>>>>>>>     > access.
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>>>>>>>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>>     > *Date: *Friday, April 7, 2017 at 17:16
>>>>>>>>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>>>>> semantics
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > Hi Arun,
>>>>>>>>     >
>>>>>>>>     > I was thinking along the same lines as you, listing the use
>>>>>>>> cases
>>>>>>>>
>>>>>>>> on the
>>>>>>>>
>>>>>>>>     > wiki, but didn't find time to get around doing that yet.
>>>>>>>>     > Don't mind if you do it if you have access now.
>>>>>>>>     > I was thinking it would be nice if, once we have the use
>>>>>>>> cases
>>>>>>>>
>>>>>>>> listed,
>>>>>>>>
>>>>>>>>     > people could use likes to up-vote the use cases similar to
>>>>>>>> what
>>>>>>>>
>>>>>>>> they're
>>>>>>>>
>>>>>>>>     > working on.
>>>>>>>>     >
>>>>>>>>     > I should have a bit more time to action this in the next
>>>>>>>> few days,
>>>>>>>>
>>>>>>>> but
>>>>>>>>
>>>>>>>>     > happy for you to do it if you can beat me to it ;-)
>>>>>>>>     >
>>>>>>>>     > Cheers,
>>>>>>>>     > Michal
>>>>>>>>     >
>>>>>>>>     > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>>>>     >
>>>>>>>>     > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > Of course. I was thinking of a subpage where people can
>>>>>>>>
>>>>>>>> collaborate.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > Will do as per Michael’s suggestion.
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > Regards,
>>>>>>>>     >
>>>>>>>>     > Arun Mathew
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>>> <
>>>>>>>>
>>>>>>>> matthias@confluent.io> wrote:
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >     Please share your Wiki-ID and a committer can give you
>>>>>>>> write
>>>>>>>>
>>>>>>>> access.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >     Btw: as you did not initiate the KIP, you should not
>>>>>>>> change the
>>>>>>>>
>>>>>>>> KIP
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     without the permission of the original author -- in
>>>>>>>> this case
>>>>>>>>
>>>>>>>> Michael.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >     So you might also just share your thought over the
>>>>>>>> mailing list
>>>>>>>>
>>>>>>>> and
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     Michael can update the KIP page. Or, as an alternative,
>>>>>>>> just
>>>>>>>>
>>>>>>>> create a
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     subpage for the KIP page.
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >     @Michael: WDYT?
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >     -Matthias
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>>>>     >
>>>>>>>>     >     > Hi Jay,
>>>>>>>>     >
>>>>>>>>     >     >           Thanks for the advise, I would like to list
>>>>>>>> down
>>>>>>>>
>>>>>>>> the use cases as
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > per your suggestion. But it seems I don't have write
>>>>>>>>
>>>>>>>> permission to the
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > Apache Kafka Confluent Space. Whom shall I request
>>>>>>>> for it?
>>>>>>>>     >
>>>>>>>>     >     >
>>>>>>>>     >
>>>>>>>>     >     > Regarding your last question. We are using a patch in
>>>>>>>> our
>>>>>>>>
>>>>>>>> production system
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > which does exactly this.
>>>>>>>>     >
>>>>>>>>     >     > We window by the event time, but trigger punctuate in
>>>>>>>>
>>>>>>>> <punctuate interval>
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > duration of system time, in the absence of an event
>>>>>>>> crossing
>>>>>>>>
>>>>>>>> the punctuate
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > event time.
>>>>>>>>     >
>>>>>>>>     >     >
>>>>>>>>     >
>>>>>>>>     >     > We are using Kafka Stream for our Audit Trail, where
>>>>>>>> we need
>>>>>>>>
>>>>>>>> to output the
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > event counts on each topic on each cluster aggregated
>>>>>>>> over a
>>>>>>>>
>>>>>>>> 1 minute
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > window. We have to use event time to be able to cross
>>>>>>>> check
>>>>>>>>
>>>>>>>> the counts. But
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > we need to trigger punctuate [aggregate event pushes]
>>>>>>>> by
>>>>>>>>
>>>>>>>> system time in the
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > absence of events. Otherwise the event counts for
>>>>>>>> unexpired
>>>>>>>>
>>>>>>>> windows would
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > be 0 which is bad.
>>>>>>>>     >
>>>>>>>>     >     >
>>>>>>>>     >
>>>>>>>>     >     > "Maybe a hybrid solution works: I window by event
>>>>>>>> time but
>>>>>>>>
>>>>>>>> trigger results
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > by system time for windows that have updated? Not
>>>>>>>> really sure
>>>>>>>>
>>>>>>>> the details
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > of making that work. Does that work? Are there
>>>>>>>> concrete
>>>>>>>>
>>>>>>>> examples where you
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > actually want the current behavior?"
>>>>>>>>     >
>>>>>>>>     >     >
>>>>>>>>     >
>>>>>>>>     >     > --
>>>>>>>>     >
>>>>>>>>     >     > With Regards,
>>>>>>>>     >
>>>>>>>>     >     >
>>>>>>>>     >
>>>>>>>>     >     > Arun Mathew
>>>>>>>>     >
>>>>>>>>     >     > Yahoo! JAPAN Corporation
>>>>>>>>     >
>>>>>>>>     >     >
>>>>>>>>     >
>>>>>>>>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>>>>>
>>>>>>>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >
>>>>>>>>     >
>>>>>>>>     >     >> Hi Jay,
>>>>>>>>     >
>>>>>>>>     >     >>
>>>>>>>>     >
>>>>>>>>     >     >> The hybrid solution is exactly what I expect and
>>>>>>>> need for
>>>>>>>>
>>>>>>>> our use cases
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> when dealing with telecom data.
>>>>>>>>     >
>>>>>>>>     >     >>
>>>>>>>>     >
>>>>>>>>     >     >> Thanks
>>>>>>>>     >
>>>>>>>>     >     >> Tianji
>>>>>>>>     >
>>>>>>>>     >     >>
>>>>>>>>     >
>>>>>>>>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>>>>>
>>>>>>>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>
>>>>>>>>     >
>>>>>>>>     >     >>> Hey guys,
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>> One thing I've always found super important for
>>>>>>>> this kind
>>>>>>>>
>>>>>>>> of design work
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> is
>>>>>>>>     >
>>>>>>>>     >     >>> to do a really good job of cataloging the landscape
>>>>>>>> of use
>>>>>>>>
>>>>>>>> cases and how
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> prevalent each one is. By that I mean not just
>>>>>>>> listing lots
>>>>>>>>
>>>>>>>> of uses, but
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> also grouping them into categories that
>>>>>>>> functionally need
>>>>>>>>
>>>>>>>> the same thing.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> In the absence of this it is very hard to reason
>>>>>>>> about
>>>>>>>>
>>>>>>>> design proposals.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> From the proposals so far I think we have a lot of
>>>>>>>>
>>>>>>>> discussion around
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> possible apis, but less around what the user needs
>>>>>>>> for
>>>>>>>>
>>>>>>>> different use
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> cases
>>>>>>>>     >
>>>>>>>>     >     >>> and how they would implement that using the api.
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>> Here is an example:
>>>>>>>>     >
>>>>>>>>     >     >>> You aggregate click and impression data for a
>>>>>>>> reddit like
>>>>>>>>
>>>>>>>> site. Every ten
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> minutes you want to output a ranked list of the top
>>>>>>>> 10
>>>>>>>>
>>>>>>>> articles ranked by
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> clicks/impressions for each geographical area. I
>>>>>>>> want to be
>>>>>>>>
>>>>>>>> able run this
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> in steady state as well as rerun to regenerate
>>>>>>>> results (or
>>>>>>>>
>>>>>>>> catch up if it
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> crashes).
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>> There are a couple of tricky things that seem to
>>>>>>>> make this
>>>>>>>>
>>>>>>>> hard with
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> either
>>>>>>>>     >
>>>>>>>>     >     >>> of the options proposed:
>>>>>>>>     >
>>>>>>>>     >     >>> 1. If I emit this data using event time I have the
>>>>>>>> problem
>>>>>>>>
>>>>>>>> described
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> where
>>>>>>>>     >
>>>>>>>>     >     >>> a geographical region with no new clicks or
>>>>>>>> impressions
>>>>>>>>
>>>>>>>> will fail to
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> output
>>>>>>>>     >
>>>>>>>>     >     >>> results.
>>>>>>>>     >
>>>>>>>>     >     >>> 2. If I emit this data using system time I have the
>>>>>>>> problem
>>>>>>>>
>>>>>>>> that when
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> reprocessing data my window may not be ten minutes
>>>>>>>> but 10
>>>>>>>>
>>>>>>>> hours if my
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> processing is very fast so it dramatically changes
>>>>>>>> the
>>>>>>>>
>>>>>>>> output.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>> Maybe a hybrid solution works: I window by event
>>>>>>>> time but
>>>>>>>>
>>>>>>>> trigger results
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> by system time for windows that have updated? Not
>>>>>>>> really
>>>>>>>>
>>>>>>>> sure the details
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> of making that work. Does that work? Are there
>>>>>>>> concrete
>>>>>>>>
>>>>>>>> examples where
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> you
>>>>>>>>     >
>>>>>>>>     >     >>> actually want the current behavior?
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>> -Jay
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>>>>>
>>>>>>>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> wrote:
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>>> Hi All,
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> Thanks for the KIP. We were also in need of a
>>>>>>>> mechanism to
>>>>>>>>
>>>>>>>> trigger
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> punctuate in the absence of events.
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> As I described in [
>>>>>>>>     >
>>>>>>>>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>>>>     >
>>>>>>>>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>>>>     >
>>>>>>>>     >     >>>> plugin.system.issuetabpanels:comment-
>>>>>>>> tabpanel#comment-
>>>>>>>>
>>>>>>>> 15926036
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> ],
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    - Our approached involved using the event time
>>>>>>>> by
>>>>>>>>
>>>>>>>> default.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    - The method to check if there is any punctuate
>>>>>>>> ready
>>>>>>>>
>>>>>>>> in the
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    PunctuationQueue is triggered via the any event
>>>>>>>>
>>>>>>>> received by the
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> stream
>>>>>>>>     >
>>>>>>>>     >     >>>>    tread, or at the polling intervals in the
>>>>>>>> absence of
>>>>>>>>
>>>>>>>> any events.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    - When we create Punctuate objects (which
>>>>>>>> contains the
>>>>>>>>
>>>>>>>> next event
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> time
>>>>>>>>     >
>>>>>>>>     >     >>>>    for punctuation and interval), we also record
>>>>>>>> the
>>>>>>>>
>>>>>>>> creation time
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> (system
>>>>>>>>     >
>>>>>>>>     >     >>>>    time).
>>>>>>>>     >
>>>>>>>>     >     >>>>    - While checking for maturity of Punctuate
>>>>>>>> Schedule by
>>>>>>>>     >
>>>>>>>>     >     >> mayBePunctuate
>>>>>>>>     >
>>>>>>>>     >     >>>>    method, we also check if the system clock has
>>>>>>>> elapsed
>>>>>>>>
>>>>>>>> the punctuate
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    interval since the schedule creation time.
>>>>>>>>     >
>>>>>>>>     >     >>>>    - In the absence of any event, or in the
>>>>>>>> absence of any
>>>>>>>>
>>>>>>>> event for
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> one
>>>>>>>>     >
>>>>>>>>     >     >>>>    topic in the partition group assigned to the
>>>>>>>> stream
>>>>>>>>
>>>>>>>> task, the system
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> time
>>>>>>>>     >
>>>>>>>>     >     >>>>    will elapse the interval and we trigger a
>>>>>>>> punctuate
>>>>>>>>
>>>>>>>> using the
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> expected
>>>>>>>>     >
>>>>>>>>     >     >>>>    punctuation event time.
>>>>>>>>     >
>>>>>>>>     >     >>>>    - we then create the next punctuation schedule
>>>>>>>> as
>>>>>>>>
>>>>>>>> punctuation event
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> time
>>>>>>>>     >
>>>>>>>>     >     >>>>    + punctuation interval, [again recording the
>>>>>>>> system
>>>>>>>>
>>>>>>>> time of creation
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> of
>>>>>>>>     >
>>>>>>>>     >     >>>> the
>>>>>>>>     >
>>>>>>>>     >     >>>>    schedule].
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>>>>> approach
>>>>>>>>
>>>>>>>> has pros and
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> cons.
>>>>>>>>     >
>>>>>>>>     >     >>>> Pros
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    - Punctuates will happen in <punctuate
>>>>>>>> interval> time
>>>>>>>>
>>>>>>>> duration at
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> max
>>>>>>>>     >
>>>>>>>>     >     >>> in
>>>>>>>>     >
>>>>>>>>     >     >>>>    terms of system time.
>>>>>>>>     >
>>>>>>>>     >     >>>>    - The semantics as a whole continues to revolve
>>>>>>>> around
>>>>>>>>
>>>>>>>> event time.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    - We can use the old data [old timestamps] to
>>>>>>>> rerun any
>>>>>>>>
>>>>>>>> experiments
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> or
>>>>>>>>     >
>>>>>>>>     >     >>>>    tests.
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> Cons
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    - In case the  <punctuate interval> is not a
>>>>>>>> time
>>>>>>>>
>>>>>>>> duration [say
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> logical
>>>>>>>>     >
>>>>>>>>     >     >>>>    time/event count], then the approach might not
>>>>>>>> be
>>>>>>>>
>>>>>>>> meaningful.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>    - In case there is a case where we have to wait
>>>>>>>> for an
>>>>>>>>
>>>>>>>> actual event
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> from
>>>>>>>>     >
>>>>>>>>     >     >>>>    a low event rate partition in the partition
>>>>>>>> group, this
>>>>>>>>
>>>>>>>> approach
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> will
>>>>>>>>     >
>>>>>>>>     >     >>>> jump
>>>>>>>>     >
>>>>>>>>     >     >>>>    the gun.
>>>>>>>>     >
>>>>>>>>     >     >>>>    - in case the event processing cannot catch up
>>>>>>>> with the
>>>>>>>>
>>>>>>>> event rate
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> and
>>>>>>>>     >
>>>>>>>>     >     >>>>    the expected timestamp events gets queued for
>>>>>>>> long
>>>>>>>>
>>>>>>>> time, this
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> approach
>>>>>>>>     >
>>>>>>>>     >     >>>>    might jump the gun.
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> I believe the above approach and discussion goes
>>>>>>>> close to
>>>>>>>>
>>>>>>>> the approach
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> A.
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> -----------
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> I like the idea of having an even count based
>>>>>>>> punctuate.
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> -----------
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> I agree with the discussion around approach C,
>>>>>>>> that we
>>>>>>>>
>>>>>>>> should provide
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> the
>>>>>>>>     >
>>>>>>>>     >     >>>> user with the option to choose system time or
>>>>>>>> event time
>>>>>>>>
>>>>>>>> based
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> punctuates.
>>>>>>>>     >
>>>>>>>>     >     >>>> But I believe that the user predominantly wants to
>>>>>>>> use
>>>>>>>>
>>>>>>>> event time while
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> not
>>>>>>>>     >
>>>>>>>>     >     >>>> missing out on regular punctuates due to event
>>>>>>>> delays or
>>>>>>>>
>>>>>>>> event
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> absences.
>>>>>>>>     >
>>>>>>>>     >     >>>> Hence a complex punctuate option as Matthias
>>>>>>>> mentioned
>>>>>>>>
>>>>>>>> (quoted below)
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> would
>>>>>>>>     >
>>>>>>>>     >     >>>> be most apt.
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> "- We might want to add "complex" schedules later
>>>>>>>> on
>>>>>>>>
>>>>>>>> (like, punctuate
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> on
>>>>>>>>     >
>>>>>>>>     >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>>>>> time
>>>>>>>>
>>>>>>>> whatever comes
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> first)."
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> -----------
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> I think I read somewhere that Kafka Streams
>>>>>>>> started with
>>>>>>>>
>>>>>>>> System Time as
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> the
>>>>>>>>     >
>>>>>>>>     >     >>>> punctuation standard, but was later changed to
>>>>>>>> Event Time.
>>>>>>>>
>>>>>>>> I guess
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> there
>>>>>>>>     >
>>>>>>>>     >     >>>> would be some good reason behind it. As Kafka
>>>>>>>> Streams want
>>>>>>>>
>>>>>>>> to evolve
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> more
>>>>>>>>     >
>>>>>>>>     >     >>>> on the Stream Processing front, I believe the
>>>>>>>> emphasis on
>>>>>>>>
>>>>>>>> event time
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> would
>>>>>>>>     >
>>>>>>>>     >     >>>> remain quite strong.
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> With Regards,
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> Arun Mathew
>>>>>>>>     >
>>>>>>>>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>>>>>
>>>>>>>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> wrote:
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> Yeah I like PuncutationType much better; I just
>>>>>>>> threw
>>>>>>>>
>>>>>>>> Time out there
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>>>>> still
>>>>>>>>
>>>>>>>> think it's
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> worth considering what this buys us over an
>>>>>>>> additional
>>>>>>>>
>>>>>>>> callback. I
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> foresee a number of punctuate implementations
>>>>>>>> following
>>>>>>>>
>>>>>>>> this pattern:
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> public void punctuate(PunctuationType type) {
>>>>>>>>     >
>>>>>>>>     >     >>>>>     switch (type) {
>>>>>>>>     >
>>>>>>>>     >     >>>>>         case EVENT_TIME:
>>>>>>>>     >
>>>>>>>>     >     >>>>>             methodA();
>>>>>>>>     >
>>>>>>>>     >     >>>>>             break;
>>>>>>>>     >
>>>>>>>>     >     >>>>>         case SYSTEM_TIME:
>>>>>>>>     >
>>>>>>>>     >     >>>>>             methodB();
>>>>>>>>     >
>>>>>>>>     >     >>>>>             break;
>>>>>>>>     >
>>>>>>>>     >     >>>>>     }
>>>>>>>>     >
>>>>>>>>     >     >>>>> }
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> I guess one advantage of this approach is we
>>>>>>>> could add
>>>>>>>>
>>>>>>>> additional
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> punctuation types later in a backwards compatible
>>>>>>>> way
>>>>>>>>
>>>>>>>> (like event
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> count
>>>>>>>>     >
>>>>>>>>     >     >>>>> as you mentioned).
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> -Tommy
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>>>>> Sax wrote:
>>>>>>>>     >
>>>>>>>>     >     >>>>>> That sounds promising.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>> I am just wondering if `Time` is the best name.
>>>>>>>> Maybe we
>>>>>>>>
>>>>>>>> want to
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> add
>>>>>>>>     >
>>>>>>>>     >     >>>>>> other non-time based punctuations at some point
>>>>>>>> later. I
>>>>>>>>
>>>>>>>> would
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>> suggest
>>>>>>>>     >
>>>>>>>>     >     >>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>> enum PunctuationType {
>>>>>>>>     >
>>>>>>>>     >     >>>>>>   EVENT_TIME,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>   SYSTEM_TIME,
>>>>>>>>     >
>>>>>>>>     >     >>>>>> }
>>>>>>>>     >
>>>>>>>>     >     >>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>> or similar. Just to keep the door open -- it's
>>>>>>>> easier to
>>>>>>>>
>>>>>>>> add new
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>> stuff
>>>>>>>>     >
>>>>>>>>     >     >>>>>> if the name is more generic.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>> -Matthias
>>>>>>>>     >
>>>>>>>>     >     >>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> I agree that the framework providing and
>>>>>>>> managing the
>>>>>>>>
>>>>>>>> notion of
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> stream
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> time is valuable and not something we would
>>>>>>>> want to
>>>>>>>>
>>>>>>>> delegate to
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> the
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>>>>> separate
>>>>>>>>
>>>>>>>> callback
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> (option
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> C)
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> is that messy (it could just be a default
>>>>>>>> method with
>>>>>>>>
>>>>>>>> an empty
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> implementation), but if we wanted a single API
>>>>>>>> to
>>>>>>>>
>>>>>>>> handle both
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> cases,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> how about something like the following?
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> enum Time {
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>    STREAM,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>    CLOCK
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> }
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> Then on ProcessorContext:
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>>>>> We could
>>>>>>>>
>>>>>>>> allow
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> this
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> to
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> be called once for each value of time to mix
>>>>>>>>
>>>>>>>> approaches.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> Then the Processor API becomes:
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>>>>>
>>>>>>>> schedule resulted
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> in
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> this call.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> Thoughts?
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>>>>> Sax
>>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> I was thinking about the four options you
>>>>>>>> proposed in
>>>>>>>>
>>>>>>>> more
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> details
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> and
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> this are my thoughts:
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> (A) You argue, that users can still
>>>>>>>> "punctuate" on
>>>>>>>>
>>>>>>>> event-time
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> via
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> process(), but I am not sure if this is
>>>>>>>> possible.
>>>>>>>>
>>>>>>>> Note, that
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> users
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> only
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> get record timestamps via context.timestamp().
>>>>>>>> Thus,
>>>>>>>>
>>>>>>>> users
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> would
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> need
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> to
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> track the time progress per partition (based
>>>>>>>> on the
>>>>>>>>
>>>>>>>> partitions
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> they
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> obverse via context.partition(). (This alone
>>>>>>>> puts a
>>>>>>>>
>>>>>>>> huge burden
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> on
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> the
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> user by itself.) However, users are not
>>>>>>>> notified at
>>>>>>>>
>>>>>>>> startup
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> what
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> partitions are assigned, and user are not
>>>>>>>> notified
>>>>>>>>
>>>>>>>> when
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> partitions
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> get
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> revoked. Because this information is not
>>>>>>>> available,
>>>>>>>>
>>>>>>>> it's not
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> possible
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> to
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> "manually advance" stream-time, and thus
>>>>>>>> event-time
>>>>>>>>
>>>>>>>> punctuation
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> within
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> process() seems not to be possible -- or do
>>>>>>>> you see a
>>>>>>>>
>>>>>>>> way to
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> get
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> it
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> done? And even if, it might still be too
>>>>>>>> clumsy to
>>>>>>>>
>>>>>>>> use.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> (B) This does not allow to mix both
>>>>>>>> approaches, thus
>>>>>>>>
>>>>>>>> limiting
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> what
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> users
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> can do.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> (C) This should give all flexibility we need.
>>>>>>>> However,
>>>>>>>>
>>>>>>>> just
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> adding
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> one
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> more method seems to be a solution that is too
>>>>>>>> simple
>>>>>>>>
>>>>>>>> (cf my
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> comments
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> below).
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>>>>> sure how
>>>>>>>>
>>>>>>>> a user
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> could
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> enable system-time and event-time punctuation
>>>>>>>> in
>>>>>>>>
>>>>>>>> parallel.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> Overall options (C) seems to be the most
>>>>>>>> promising
>>>>>>>>
>>>>>>>> approach to
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> me.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> Because I also favor a clean API, we might
>>>>>>>> keep
>>>>>>>>
>>>>>>>> current
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> punctuate()
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>>>>> at some
>>>>>>>>
>>>>>>>> later
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> point
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> when
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> people use the "new punctuate API".
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> Couple of follow up questions:
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> - I am wondering, if we should have two
>>>>>>>> callback
>>>>>>>>
>>>>>>>> methods or
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> just
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> one
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> (ie, a unified for system and event time
>>>>>>>> punctuation
>>>>>>>>
>>>>>>>> or one for
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> each?).
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> - If we have one, how can the user figure out,
>>>>>>>> which
>>>>>>>>
>>>>>>>> condition
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> did
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> trigger?
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> - How would the API look like, for registering
>>>>>>>>
>>>>>>>> different
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> punctuate
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> - We might want to add "complex" schedules
>>>>>>>> later on
>>>>>>>>
>>>>>>>> (like,
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> punctuate
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> on
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>>>>> system-time
>>>>>>>>
>>>>>>>> whatever
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> comes
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> first). I don't say we should add this right
>>>>>>>> away, but
>>>>>>>>
>>>>>>>> we might
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> want
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> to
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> define the API in a way, that it allows
>>>>>>>> extensions
>>>>>>>>
>>>>>>>> like this
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> later
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> on,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> without redesigning the API (ie, the API
>>>>>>>> should be
>>>>>>>>
>>>>>>>> designed
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> extensible)
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> - Did you ever consider count-based
>>>>>>>> punctuation?
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> I understand, that you would like to solve a
>>>>>>>> simple
>>>>>>>>
>>>>>>>> problem,
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> but
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> we
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> learned from the past, that just "adding some
>>>>>>>> API"
>>>>>>>>
>>>>>>>> quickly
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> leads
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> to a
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> not very well defined API that needs time
>>>>>>>> consuming
>>>>>>>>
>>>>>>>> clean up
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> later on
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>>>>> holistic
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> punctuation
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> KIP
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> with this from the beginning on to avoid later
>>>>>>>> painful
>>>>>>>>     >
>>>>>>>>     >     >> redesign.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> -Matthias
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> Thanks Thomas,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> I'm also wary of changing the existing
>>>>>>>> semantics of
>>>>>>>>     >
>>>>>>>>     >     >> punctuate,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> for
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> backward compatibility reasons, although I
>>>>>>>> like the
>>>>>>>>     >
>>>>>>>>     >     >> conceptual
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> simplicity of that option.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>>>>> a way,
>>>>>>>>
>>>>>>>> uglier.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> I
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> added
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> this to the KIP now as option (C).
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>>>>> more
>>>>>>>>
>>>>>>>> flexible,
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> as
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> it
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> allows
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> you to return any value, you're not limited
>>>>>>>> to event
>>>>>>>>
>>>>>>>> time or
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> system
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> time
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> (although I don't see an actual use case
>>>>>>>> where you
>>>>>>>>
>>>>>>>> might need
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> anything
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> else then those two). Hence I also proposed
>>>>>>>> the
>>>>>>>>
>>>>>>>> option to
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> allow
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> users
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>>>>> for
>>>>>>>>
>>>>>>>> them given
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> the
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> presence or absence of messages, much like
>>>>>>>> they can
>>>>>>>>
>>>>>>>> decide
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> what
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> msg
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> time
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>>>>> What do
>>>>>>>>
>>>>>>>> you
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> think
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> about
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> that? This is probably most flexible but also
>>>>>>>> most
>>>>>>>>     >
>>>>>>>>     >     >> complicated.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> All comments appreciated.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> Cheers,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> Michal
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> Although I fully agree we need a way to
>>>>>>>> trigger
>>>>>>>>
>>>>>>>> periodic
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> processing
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> that is independent from whether and when
>>>>>>>> messages
>>>>>>>>
>>>>>>>> arrive,
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> I'm
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> not sure
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> I like the idea of changing the existing
>>>>>>>> semantics
>>>>>>>>
>>>>>>>> across
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> the
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> board.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> What if we added an additional callback to
>>>>>>>> Processor
>>>>>>>>
>>>>>>>> that
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> can
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> be
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>>>>> always
>>>>>>>>
>>>>>>>> called at
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> fixed, wall
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>>>>> have to
>>>>>>>>
>>>>>>>> give
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> up
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> the
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> notion
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> of stream time to be able to do periodic
>>>>>>>> processing.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>>>>> Borowiecki
>>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> Hi all,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>>>>>
>>>>>>>> punctuate
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> semantics
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>>>>>
>>>>>>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/ confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>>>>>
>>>>>>>> 138%
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> 3A+C
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> hange+
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> punctuate+semantics>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> .
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> Appreciating there can be different views
>>>>>>>> on
>>>>>>>>
>>>>>>>> system-time
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> vs
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> event-
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> time
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>>>>> case and
>>>>>>>>
>>>>>>>> the
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> importance of
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>>>>> I've
>>>>>>>>
>>>>>>>> left it
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> quite
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> open
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> and
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>>>>>
>>>>>>>> progresses.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> Thanks,
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>>>>> Michal
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> --
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>     Tommy Becker
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>     Senior Software Engineer
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>     tivo.com
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> ________________________________
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> This email and any attachments may contain
>>>>>>>> confidential
>>>>>>>>
>>>>>>>> and
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> privileged material for the sole use of the
>>>>>>>> intended
>>>>>>>>
>>>>>>>> recipient.
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> Any
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> review, copying, or distribution of this email
>>>>>>>> (or any
>>>>>>>>     >
>>>>>>>>     >     >> attachments)
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> by others is prohibited. If you are not the
>>>>>>>> intended
>>>>>>>>
>>>>>>>> recipient,
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> please contact the sender immediately and
>>>>>>>> permanently
>>>>>>>>
>>>>>>>> delete this
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> email and any attachments. No employee or agent
>>>>>>>> of TiVo
>>>>>>>>
>>>>>>>> Inc. is
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> authorized to conclude any binding agreement on
>>>>>>>> behalf
>>>>>>>>
>>>>>>>> of TiVo
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> Inc.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>>>>> only be
>>>>>>>>
>>>>>>>> made by a
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>>> signed written agreement.
>>>>>>>>     >
>>>>>>>>     >     >>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> --
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>     Tommy Becker
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>     Senior Software Engineer
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>     tivo.com
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> ________________________________
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> This email and any attachments may contain
>>>>>>>> confidential
>>>>>>>>
>>>>>>>> and
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> privileged
>>>>>>>>     >
>>>>>>>>     >     >>>>> material for the sole use of the intended
>>>>>>>> recipient. Any
>>>>>>>>
>>>>>>>> review,
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>> copying,
>>>>>>>>     >
>>>>>>>>     >     >>>>> or distribution of this email (or any
>>>>>>>> attachments) by
>>>>>>>>
>>>>>>>> others is
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>> prohibited.
>>>>>>>>     >
>>>>>>>>     >     >>>>> If you are not the intended recipient, please
>>>>>>>> contact the
>>>>>>>>
>>>>>>>> sender
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> immediately and permanently delete this email and
>>>>>>>> any
>>>>>>>>
>>>>>>>> attachments. No
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>>>>> conclude
>>>>>>>>
>>>>>>>> any binding
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>>>>> Binding
>>>>>>>>
>>>>>>>> agreements with
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     >     >> TiVo
>>>>>>>>     >
>>>>>>>>     >     >>>>> Inc. may only be made by a signed written
>>>>>>>> agreement.
>>>>>>>>     >
>>>>>>>>     >     >>>>>
>>>>>>>>     >
>>>>>>>>     >     >>>>
>>>>>>>>     >
>>>>>>>>     >     >>>
>>>>>>>>     >
>>>>>>>>     >     >>
>>>>>>>>     >
>>>>>>>>     >     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > --
>>>>>>>>     >
>>>>>>>>     > <http://www.openbet.com/> <http://www.openbet.com/>
>>>>>>>>
>>>>>>>>     >
>>>>>>>>     > *Michal Borowiecki*
>>>>>>>>     >
>>>>>>>>     > *Senior Software Engineer L4*
>>>>>>>>     >
>>>>>>>>     > *T: *
>>>>>>>>     >
>>>>>>>>     > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>>>>>     >
>>>>>>>>     > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > *E: *
>>>>>>>>     >
>>>>>>>>     > michal.borowiecki@openbet.com
>>>>>>>>     >
>>>>>>>>     > *W: *
>>>>>>>>     >
>>>>>>>>     > www.openbet.com
>>>>>>>>     >
>>>>>>>>     > *OpenBet Ltd*
>>>>>>>>     >
>>>>>>>>     > Chiswick Park Building 9
>>>>>>>>     >
>>>>>>>>     > 566 Chiswick High Rd
>>>>>>>>     >
>>>>>>>>     > London
>>>>>>>>     >
>>>>>>>>     > W4 5XT
>>>>>>>>     >
>>>>>>>>     > UK
>>>>>>>>     >
>>>>>>>>     > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     > This message is confidential and intended only for the
>>>>>>>> addressee.
>>>>>>>>
>>>>>>>> If you
>>>>>>>>
>>>>>>>>     > have received this message in error, please immediately
>>>>>>>> notify the
>>>>>>>>     > postmaster@openbet.com and delete it from your system as
>>>>>>>> well as
>>>>>>>>
>>>>>>>> any
>>>>>>>>
>>>>>>>>     > copies. The content of e-mails as well as traffic data may
>>>>>>>> be
>>>>>>>>
>>>>>>>> monitored by
>>>>>>>>
>>>>>>>>     > OpenBet for employment and security purposes. To protect
>>>>>>>> the
>>>>>>>>
>>>>>>>> environment
>>>>>>>>
>>>>>>>>     > please do not print this e-mail unless necessary. OpenBet
>>>>>>>> Ltd.
>>>>>>>>
>>>>>>>> Registered
>>>>>>>>
>>>>>>>>     > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>>>>> London,
>>>>>>>>
>>>>>>>> W4 5XT,
>>>>>>>>
>>>>>>>>     > United Kingdom. A company registered in England and Wales.
>>>>>>>>
>>>>>>>> Registered no.
>>>>>>>>
>>>>>>>>     > 3134634. VAT no. GB927523612
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>     >
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>>
>>>>>>>>
>>>>>>>>     Tommy Becker
>>>>>>>>
>>>>>>>>     Senior Software Engineer
>>>>>>>>
>>>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747>
>>>>>>>>
>>>>>>>>
>>>>>>>>     tivo.com
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> ________________________________
>>>>>>>>
>>>>>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> <http://www.openbet.com/> Michal Borowiecki
>>>>>>>> Senior Software Engineer L4
>>>>>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>> -- 
>>>>> Signature
>>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>>> Senior Software Engineer L4
>>>>> 	T: 	+44 208 742 1600
>>>>>
>>>>> 	
>>>>> 	+44 203 249 8448
>>>>>
>>>>> 	
>>>>> 	 
>>>>> 	E: 	michal.borowiecki@openbet.com
>>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>>
>>>>> 	
>>>>> 	OpenBet Ltd
>>>>>
>>>>> 	Chiswick Park Building 9
>>>>>
>>>>> 	566 Chiswick High Rd
>>>>>
>>>>> 	London
>>>>>
>>>>> 	W4 5XT
>>>>>
>>>>> 	UK
>>>>>
>>>>> 	
>>>>> <https://www.openbet.com/email_promo>
>>>>>
>>>>> This message is confidential and intended only for the addressee. If
>>>>> you have received this message in error, please immediately notify the
>>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>>> from your system as well as any copies. The content of e-mails as well
>>>>> as traffic data may be monitored by OpenBet for employment and
>>>>> security purposes. To protect the environment please do not print this
>>>>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>>>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>>>> company registered in England and Wales. Registered no. 3134634. VAT
>>>>> no. GB927523612
>>>>>
>>>> -- 
>>>> Signature
>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>> Senior Software Engineer L4
>>>> 	T: 	+44 208 742 1600
>>>>
>>>> 	
>>>> 	+44 203 249 8448
>>>>
>>>> 	
>>>> 	 
>>>> 	E: 	michal.borowiecki@openbet.com
>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>
>>>> 	
>>>> 	OpenBet Ltd
>>>>
>>>> 	Chiswick Park Building 9
>>>>
>>>> 	566 Chiswick High Rd
>>>>
>>>> 	London
>>>>
>>>> 	W4 5XT
>>>>
>>>> 	UK
>>>>
>>>> 	
>>>> <https://www.openbet.com/email_promo>
>>>>
>>>> This message is confidential and intended only for the addressee. If you
>>>> have received this message in error, please immediately notify the
>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>> from your system as well as any copies. The content of e-mails as well
>>>> as traffic data may be monitored by OpenBet for employment and security
>>>> purposes. To protect the environment please do not print this e-mail
>>>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>>> registered in England and Wales. Registered no. 3134634. VAT no.
>>>> GB927523612
>>>>
>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	 
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If
>> you have received this message in error, please immediately notify the
>> postmaster@openbet.com <ma...@openbet.com> and delete it
>> from your system as well as any copies. The content of e-mails as well
>> as traffic data may be monitored by OpenBet for employment and
>> security purposes. To protect the environment please do not print this
>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>> company registered in England and Wales. Registered no. 3134634. VAT
>> no. GB927523612
>>
> 
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
> 
> 	
> 	+44 203 249 8448
> 
> 	
> 	 
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
> 
> 	
> 	OpenBet Ltd
> 
> 	Chiswick Park Building 9
> 
> 	566 Chiswick High Rd
> 
> 	London
> 
> 	W4 5XT
> 
> 	UK
> 
> 	
> <https://www.openbet.com/email_promo>
> 
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com <ma...@openbet.com> and delete it
> from your system as well as any copies. The content of e-mails as well
> as traffic data may be monitored by OpenBet for employment and security
> purposes. To protect the environment please do not print this e-mail
> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> registered in England and Wales. Registered no. 3134634. VAT no.
> GB927523612
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Further to this, on your point about configuration:

> Thus, I also believe that one might need different "configuration"
> values for the hybrid approach if you run the same code for different
> scenarios: regular processing, re-processing, catching up scenario. And
> as the term "configuration" implies, we might be better off to not mix
> configuration with business logic that is expressed via code.
I'm not sure I understand what you are suggesting here.

Configuration is global to a KafkaStreams instance and users might want 
to have different tolerance in different parts of the topology. They 
shouldn't be locked into one value set via global config.

To illustrate this point: Lately I have discovered the cache config 
introduced in KIP-63 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-63%3A+Unify+store+and+downstream+caching+in+streams> 
and found it quite annoying that it's controlled by a config item. IMO, 
I should be able to control flushing per processor, not be forced to use 
one global value defined in configs.

It's easy enough for users to source a user-defined config and provided 
it as a parameter to a /given /processor as needed.

In principal I agree that configuration and business logic are better 
not mixed together but then the configuration mechanism should allow 
users to target specific processors and not be global to the 
KafkaStreams instance.

Thanks,

Michal

On 24/04/17 10:23, Michal Borowiecki wrote:
>
> Hi Matthias,
>
> I agree it's difficult to reason about the hybrid approach, I 
> certainly found it hard and I'm totally on board with the mantra.
>
> I'd be happy to limit the scope of this KIP to add system-time 
> punctuation semantics (in addition to existing stream-time semantics) 
> and leave more complex schemes for users to implement on top of that.
>
> Further additional PunctuationTypes, could then be added by future 
> KIPs, possibly including the hybrid approach once it has been given 
> more thought.
>
>> There are real-time applications, that want to get
>> callbacks in regular system-time intervals (completely independent from
>> stream-time).
> Can you please describe what they are, so that I can put them on the 
> wiki for later reference?
>
> Thanks,
>
> Michal
>
>
> On 23/04/17 21:27, Matthias J. Sax wrote:
>> Hi,
>>
>> I do like Damian's API proposal about the punctuation callback function.
>>
>> I also did reread the KIP and thought about the semantics we want to
>> provide.
>>
>>> Given the above, I don't see a reason any more for a separate system-time based punctuation.
>> I disagree here. There are real-time applications, that want to get
>> callbacks in regular system-time intervals (completely independent from
>> stream-time). Thus we should allow this -- if we really follow the
>> "hybrid" approach, this could be configured with stream-time interval
>> infinite and delay whatever system-time punctuation interval you want to
>> have. However, I would like to add a proper API for this and do this
>> configuration under the hood (that would allow one implementation within
>> all kind of branching for different cases).
>>
>> Thus, we definitely should have PunctutionType#StreamTime and
>> #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
>> a fan of your latest API proposal.
>>
>>
>> About the hybrid approach in general. On the one hand I like it, on the
>> other hand, it seems to be rather (1) complicated (not necessarily from
>> an implementation point of view, but for people to understand it) and
>> (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
>>
>>> It may appear complicated at first but I do think these semantics will
>>> still be more understandable to users than having 2 separate punctuation
>>> schedules/callbacks with different PunctuationTypes.
>> This statement only holds if you apply strong assumptions that I don't
>> believe hold in general -- see (2) for details -- and I think it is
>> harder than you assume to reason about the hybrid approach in general.
>> IMHO, the hybrid approach is a "false friend" that seems to be easy to
>> reason about...
>>
>>
>> (1) Streams always embraced "easy to use" and we should really be
>> careful to keep it this way. On the other hand, as we are talking about
>> changes to PAPI, it won't affect DSL users (DSL does not use punctuation
>> at all at the moment), and thus, the "easy to use" mantra might not be
>> affected, while it will allow advanced users to express more complex stuff.
>>
>> I like the mantra: "make simple thing easy and complex things possible".
>>
>> (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
>> implicit assumption that even-time progresses at the same "speed" as
>> system-time during regular processing. This implies the assumption that
>> a slower progress in stream-time indicates the absence of input events
>> (and that later arriving input events will have a larger event-time with
>> high probability). Even if this might be true for some use cases, I
>> doubt it holds in general. Assume that you get a spike in traffic and
>> for some reason stream-time does advance slowly because you have more
>> records to process. This might trigger a system-time based punctuation
>> call even if this seems not to be intended. I strongly believe that it
>> is not easy to reason about the semantics of the hybrid approach (even
>> if the intentional semantics would be super useful -- but I doubt that
>> we get want we ask for).
>>
>> Thus, I also believe that one might need different "configuration"
>> values for the hybrid approach if you run the same code for different
>> scenarios: regular processing, re-processing, catching up scenario. And
>> as the term "configuration" implies, we might be better off to not mix
>> configuration with business logic that is expressed via code.
>>
>>
>> One more comment: I also don't think that the hybrid approach is
>> deterministic as claimed in the use-case subpage. I understand the
>> reasoning and agree, that it is deterministic if certain assumptions
>> hold -- compare above -- and if configured correctly. But strictly
>> speaking it's not because there is a dependency on system-time (and
>> IMHO, if system-time is involved it cannot be deterministic by definition).
>>
>>
>>> I see how in theory this could be implemented on top of the 2 punctuate
>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>> the other system-time based) but it would be a much more complicated
>>> scheme and I don't want to suggest that.
>> I agree that expressing the intended hybrid semantics is harder if we
>> offer only #StreamTime and #SystemTime punctuation. However, I also
>> believe that the hybrid approach is a "false friend" with regard to
>> reasoning about the semantics (it indicates that it more easy as it is
>> in reality). Therefore, we might be better off to not offer the hybrid
>> approach and make it clear to a developed, that it is hard to mix
>> #StreamTime and #SystemTime in a semantically sound way.
>>
>>
>> Looking forward to your feedback. :)
>>
>> -Matthias
>>
>>
>>
>>
>> On 4/22/17 11:43 AM, Michal Borowiecki wrote:
>>> Hi all,
>>>
>>> Looking for feedback on the functional interface approach Damian
>>> proposed. What do people think?
>>>
>>> Further on the semantics of triggering punctuate though:
>>>
>>> I ran through the 2 use cases that Arun had kindly put on the wiki
>>> (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
>>> in my head and on a whiteboard and I can't find a better solution than
>>> the "hybrid" approach he had proposed.
>>>
>>> I see how in theory this could be implemented on top of the 2 punctuate
>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>> the other system-time based) but it would be a much more complicated
>>> scheme and I don't want to suggest that.
>>>
>>> However, to add to the hybrid algorithm proposed, I suggest one
>>> parameter to that: a tolerance period, expressed in milliseconds
>>> system-time, after which the punctuation will be invoked in case the
>>> stream-time advance hasn't triggered it within the requested interval
>>> since the last invocation of punctuate (as recorded in system-time)
>>>
>>> This would allow a user-defined tolerance for late arriving events. The
>>> trade off would be left for the user to decide: regular punctuation in
>>> the case of absence of events vs allowing for records arriving late or
>>> some build-up due to processing not catching up with the event rate.
>>> In the one extreme, this tolerance could be set to infinity, turning
>>> hybrid into simply stream-time based punctuate, like we have now. In the
>>> other extreme, the tolerance could be set to 0, resulting in a
>>> system-time upper bound on the effective punctuation interval.
>>>
>>> Given the above, I don't see a reason any more for a separate
>>> system-time based punctuation. The "hybrid" approach with 0ms tolerance
>>> would under normal operation trigger at regular intervals wrt the
>>> system-time, except in cases of re-play/catch-up, where the stream time
>>> advances faster than system time. In these cases punctuate would happen
>>> more often than the specified interval wrt system time. However, the
>>> use-cases that need system-time punctuations (that I've seen at least)
>>> really only have a need for an upper bound on punctuation delay but
>>> don't need a lower bound.
>>>
>>> To that effect I'd propose the api to be as follows, on ProcessorContext:
>>>
>>> schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
>>>
>>> schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
>>>
>>> Punctuation is triggered when either:
>>> - the stream time advances past the (stream time of the previous
>>> punctuation) + interval;
>>> - or (iff the toleranceInterval is set) when the system time advances
>>> past the (system time of the previous punctuation) + interval +
>>> toleranceInterval
>>>
>>> In either case:
>>> - we trigger punctuate passing as the argument the stream time at which
>>> the current punctuation was meant to happen
>>> - next punctuate is scheduled at (stream time at which the current
>>> punctuation was meant to happen) + interval
>>>
>>> It may appear complicated at first but I do think these semantics will
>>> still be more understandable to users than having 2 separate punctuation
>>> schedules/callbacks with different PunctuationTypes.
>>>
>>>
>>>
>>> PS. Having re-read this, maybe the following alternative would be easier
>>> to understand (WDYT?):
>>>
>>> schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
>>>
>>> schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
>>>
>>> Punctuation is triggered when either:
>>> - the stream time advances past the (stream time of the previous
>>> punctuation) + streamTimeInterval;
>>> - or (iff systemTimeUpperBound is set) when the system time advances
>>> past the (system time of the previous punctuation) + systemTimeUpperBound
>>>
>>> Awaiting comments.
>>>
>>> Thanks,
>>> Michal
>>>
>>> On 21/04/17 16:56, Michal Borowiecki wrote:
>>>> Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>>>> in favour of something that doesn't return a record. Not a problem though.
>>>>
>>>>
>>>> On 21/04/17 16:32, Damian Guy wrote:
>>>>> Thanks Michal,
>>>>> I agree Transformer.punctuate should also be void, but we can deprecate
>>>>> that too in favor of the new interface.
>>>>>
>>>>> Thanks for the javadoc PR!
>>>>>
>>>>> Cheers,
>>>>> Damian
>>>>>
>>>>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>
>>>>>> Yes, that looks better to me.
>>>>>>
>>>>>> Note that punctuate on Transformer is currently returning a record, but I
>>>>>> think it's ok to have all output records be sent via
>>>>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>>>>> multiple records from one invocation of punctuate.
>>>>>>
>>>>>> This way it's consistent between Processor and Transformer.
>>>>>>
>>>>>>
>>>>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>>>>> there:
>>>>>>
>>>>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>>>>
>>>>>> and PR:https://github.com/apache/kafka/pull/2884
>>>>>>
>>>>>> Cheers,
>>>>>>
>>>>>> Michal
>>>>>> On 20/04/17 18:55, Damian Guy wrote:
>>>>>>
>>>>>> Hi Michal,
>>>>>>
>>>>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>>>>> the API.
>>>>>> 1. deprecate the punctuate method on Processor
>>>>>> 2. create a new Functional Interface just for Punctuation, something like:
>>>>>> interface Punctuator {
>>>>>>      void punctuate(long timestamp)
>>>>>> }
>>>>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>>>>> interval, PunctuationType type, Punctuator callback)
>>>>>> 4. deprecate the existing schedule function
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>> Thanks,
>>>>>> Damian
>>>>>>
>>>>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>
>>>>>>> Hi Thomas,
>>>>>>>
>>>>>>> I would say our use cases fall in the same category as yours.
>>>>>>>
>>>>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>>>>
>>>>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>>>>> of design:
>>>>>>>
>>>>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>>>>
>>>>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>>>>
>>>>>>> Since the output field X is derived in some non-trivial way, we don't
>>>>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>>>>> another input field Z).
>>>>>>>
>>>>>>> So we have kv stores with the records and an additional kv store with
>>>>>>> timestamp->id mapping which act like an index where we periodically do a
>>>>>>> ranged query.
>>>>>>>
>>>>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>>>>> work when there were no regular msgs on the input topic.
>>>>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>>>>> outside we created a "ticker" that produced msgs once per second onto
>>>>>>> another topic and fed it into the same topology to trigger punctuate.
>>>>>>> This didn't work either, which was much more surprising to us at the
>>>>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>>>>> *all* input partitions receive messages regularly.
>>>>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>>>>> is consumed by the second topology and is its only input topic. There's a
>>>>>>> transformer on that topic which populates and updates the time-based
>>>>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>>>>> elapsed, the record id is sent to the main transformer, which
>>>>>>> updates/deletes the record from the main kv store and forwards the
>>>>>>> transformed record to the output topic.
>>>>>>>
>>>>>>> To me this setup feels horrendously complicated for what it does.
>>>>>>>
>>>>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>>>>> However, the ticks don't feel so hacky when you realise they give you
>>>>>>> some hypothetical benefits in predictability. You can reprocess the
>>>>>>> messages in a reproducible manner, since the topologies use event-time,
>>>>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>>>>> reprocessing).
>>>>>>> To make that work though, we would have to have the stream time advance
>>>>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>>>>> presence of messages on the other input topic.
>>>>>>>
>>>>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>>>>> and the hybrid would work to simplify this a lot.
>>>>>>>
>>>>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>>>>> have to go with punctuate there too.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Michal
>>>>>>>
>>>>>>>
>>>>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>>>>
>>>>>>> Here's an example that we currently have.  We have a streams processor
>>>>>>> that does a transform from one topic into another. One of the fields in
>>>>>>> the source topic record is an expiration time, and one of the functions
>>>>>>> of the processor is to ensure that expired records get deleted promptly
>>>>>>> after that time passes (typically days or weeks after the message was
>>>>>>> originally produced). To do that, the processor keeps a state store of
>>>>>>> keys and expiration times, iterates that store in punctuate(), and
>>>>>>> emits delete (null) records for expired items. This needs to happen at
>>>>>>> some minimum interval regardless of the incoming message rate of the
>>>>>>> source topic.
>>>>>>>
>>>>>>> In this scenario, the expiration of records is the primary function of
>>>>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>>>>> range of use-cases.
>>>>>>>
>>>>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>>>>
>>>>>>> I apologize for the longer email below.  To my defense, it started
>>>>>>> out much
>>>>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>>>>> devil's
>>>>>>> advocate for a number of arguments brought forth in order to help
>>>>>>> improve
>>>>>>> this KIP -- I am not implying I necessarily disagree with the
>>>>>>> arguments.
>>>>>>>
>>>>>>> That aside, here are some further thoughts.
>>>>>>>
>>>>>>> First, there are (at least?) two categories for actions/behavior you
>>>>>>> invoke
>>>>>>> via punctuate():
>>>>>>>
>>>>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>>>>> to
>>>>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>>>>> the
>>>>>>> impact of punctuate is typically not observable by other processing
>>>>>>> nodes
>>>>>>> in the topology.
>>>>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>>>>> the
>>>>>>> punctuate is all about being observable by downstream processing
>>>>>>> nodes.
>>>>>>>
>>>>>>> A few releases back, we introduced record caches (DSL) and state
>>>>>>> store
>>>>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>>>>> relating to
>>>>>>> (2) where some users needed to control -- here: limit -- the
>>>>>>> downstream
>>>>>>> output rate of Kafka Streams because the downstream systems/apps
>>>>>>> would not
>>>>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>>>>> their
>>>>>>> scalability).  The argument for KIP-63, which notably did not
>>>>>>> introduce a
>>>>>>> "trigger" API, was that such an interaction with downstream systems
>>>>>>> is an
>>>>>>> operational concern;  it should not impact the processing *logic* of
>>>>>>> your
>>>>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>>>>> API,
>>>>>>> especially not the declarative DSL, with such operational concerns.
>>>>>>>
>>>>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>>>>> sorry, I
>>>>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>>>>> observing that our conversation is moving more and more into the
>>>>>>> direction
>>>>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>>>>> for
>>>>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>>>>> comments voiced here are about sth like "IF stream-time didn't
>>>>>>> trigger
>>>>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>>>>> want
>>>>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>>>>> related
>>>>>>> note, whatever we are discussing here will impact state store caches
>>>>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>>>>> should
>>>>>>> clarify any such impact here.
>>>>>>>
>>>>>>> Switching topics slightly.
>>>>>>>
>>>>>>> Jay wrote:
>>>>>>>
>>>>>>> One thing I've always found super important for this kind of design
>>>>>>> work
>>>>>>> is to do a really good job of cataloging the landscape of use cases
>>>>>>> and
>>>>>>> how prevalent each one is.
>>>>>>>
>>>>>>> +1 to this, as others have already said.
>>>>>>>
>>>>>>> Here, let me highlight -- just in case -- that when we talked about
>>>>>>> windowing use cases in the recent emails, the Processor API (where
>>>>>>> `punctuate` resides) does not have any notion of windowing at
>>>>>>> all.  If you
>>>>>>> want to do windowing *in the Processor API*, you must do so manually
>>>>>>> in
>>>>>>> combination with window stores.  For this reason I'd suggest to
>>>>>>> discuss use
>>>>>>> cases not just in general, but also in view of how you'd do so in the
>>>>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>>>>> `punctuate`
>>>>>>> does not impact the DSL at all, unless we add new functionality to
>>>>>>> it.
>>>>>>>
>>>>>>> Jay wrote in his strawman example:
>>>>>>>
>>>>>>> You aggregate click and impression data for a reddit like site.
>>>>>>> Every ten
>>>>>>> minutes you want to output a ranked list of the top 10 articles
>>>>>>> ranked by
>>>>>>> clicks/impressions for each geographical area. I want to be able
>>>>>>> run this
>>>>>>> in steady state as well as rerun to regenerate results (or catch up
>>>>>>> if it
>>>>>>> crashes).
>>>>>>>
>>>>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>>>>> we
>>>>>>> argued that the reason for saying "every ten minutes" above is not
>>>>>>> necessarily about because you want to output data *exactly* after ten
>>>>>>> minutes, but that you want to perform an aggregation based on 10-
>>>>>>> minute
>>>>>>> windows of input data; i.e., the point is about specifying the input
>>>>>>> for
>>>>>>> your aggregation, not or less about when the results of the
>>>>>>> aggregation
>>>>>>> should be send downstream.  To take an extreme example, you could
>>>>>>> disable
>>>>>>> record caches and let your app output a downstream update for every
>>>>>>> incoming input record.  If the last input record was from at minute 7
>>>>>>> of 10
>>>>>>> (for a 10-min window), then what your app would output at minute 10
>>>>>>> would
>>>>>>> be identical to what it had already emitted at minute 7 earlier
>>>>>>> anyways.
>>>>>>> This is particularly true when we take late-arriving data into
>>>>>>> account:  if
>>>>>>> a late record arrived at minute 13, your app would (by default) send
>>>>>>> a new
>>>>>>> update downstream, even though the "original" 10 minutes have already
>>>>>>> passed.
>>>>>>>
>>>>>>> Jay wrote...:
>>>>>>>
>>>>>>> There are a couple of tricky things that seem to make this hard
>>>>>>> with
>>>>>>>
>>>>>>> either
>>>>>>>
>>>>>>> of the options proposed:
>>>>>>> 1. If I emit this data using event time I have the problem
>>>>>>> described where
>>>>>>> a geographical region with no new clicks or impressions will fail
>>>>>>> to
>>>>>>>
>>>>>>> output
>>>>>>>
>>>>>>> results.
>>>>>>>
>>>>>>> ...and Arun Mathew wrote:
>>>>>>>
>>>>>>>
>>>>>>> We window by the event time, but trigger punctuate in <punctuate
>>>>>>> interval>
>>>>>>> duration of system time, in the absence of an event crossing the
>>>>>>> punctuate
>>>>>>> event time.
>>>>>>>
>>>>>>> So, given what I wrote above about the status quo and what you can
>>>>>>> already
>>>>>>> do with it, is the concern that the state store cache doesn't give
>>>>>>> you
>>>>>>> *direct* control over "forcing an output after no later than X
>>>>>>> seconds [of
>>>>>>> processing-time]" but only indirect control through a cache
>>>>>>> size?  (Note
>>>>>>> that I am not dismissing the claims why this might be helpful.)
>>>>>>>
>>>>>>> Arun Mathew wrote:
>>>>>>>
>>>>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>>>>> output the
>>>>>>> event counts on each topic on each cluster aggregated over a 1
>>>>>>> minute
>>>>>>> window. We have to use event time to be able to cross check the
>>>>>>> counts.
>>>>>>>
>>>>>>> But
>>>>>>>
>>>>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>>>>> time in
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> absence of events. Otherwise the event counts for unexpired windows
>>>>>>> would
>>>>>>> be 0 which is bad.
>>>>>>>
>>>>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>>>>> absence
>>>>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>>>>> Scala
>>>>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>>>>> root
>>>>>>> cause that the downstream system interprets the absence of output in
>>>>>>> a
>>>>>>> particular way ("No output after 1 minute = I consider the output to
>>>>>>> be
>>>>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>>>>> possible)
>>>>>>> to correctly handle the difference between absence of output vs.
>>>>>>> output of
>>>>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>>>>> but
>>>>>>> want to understand the motivation better. :-)
>>>>>>>
>>>>>>> Also, to add some perspective, in some related discussions we talked
>>>>>>> about
>>>>>>> how a Kafka Streams application should not worry or not be coupled
>>>>>>> unnecessarily with such interpretation specifics in a downstream
>>>>>>> system's
>>>>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>>>>> more
>>>>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>>>>> than
>>>>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>>>>> and
>>>>>>> downstream systems, including helping to reconcile the differences in
>>>>>>> how
>>>>>>> these systems interpret changes, updates, late-arriving data,
>>>>>>> etc.  Kafka
>>>>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>>>>> processing from the specifics of downstream systems, thanks to
>>>>>>> specific
>>>>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>>>>> this
>>>>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>>>>> but
>>>>>>> it's currently awkward to use Connect for this", this might be a
>>>>>>> problem we
>>>>>>> can solve, too.)
>>>>>>>
>>>>>>> Switching topics slightly again.
>>>>>>>
>>>>>>> Thomas wrote:
>>>>>>>
>>>>>>> I'm not entirely convinced that a separate callback (option C)
>>>>>>> is that messy (it could just be a default method with an empty
>>>>>>> implementation), but if we wanted a single API to handle both
>>>>>>> cases,
>>>>>>> how about something like the following?
>>>>>>>
>>>>>>> enum Time {
>>>>>>>     STREAM,
>>>>>>>     CLOCK
>>>>>>> }
>>>>>>>
>>>>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>>>>> then
>>>>>>> whatever the user is doing inside this method is a black box to Kafka
>>>>>>> Streams (similar to how we have no idea what the user does inside a
>>>>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>>>>> won't
>>>>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>>>>> frequency than the processing-time action.  Or, we won't know whether
>>>>>>> the
>>>>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>>>>> minute of
>>>>>>> stream-time or 1-minute of processing-time, whichever comes
>>>>>>> first").  That
>>>>>>> said, I am not certain yet whether we would need such knowledge
>>>>>>> because,
>>>>>>> when using the Processor API, most of the work and decisions must be
>>>>>>> done
>>>>>>> by the user anyways.  It would matter though if the concept of
>>>>>>> "triggers"
>>>>>>> were to bubble up into the DSL because in the DSL the management of
>>>>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>>>>> Streams.
>>>>>>>
>>>>>>> [In any case, btw, we have the corner case where the user configured
>>>>>>> the
>>>>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>>>>> extractor), at which point both punctuate variants are based on the
>>>>>>> same
>>>>>>> time semantics / timeline.]
>>>>>>>
>>>>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>>>>> it
>>>>>>> this far. :-)
>>>>>>>
>>>>>>> More than happy to hear your thoughts!
>>>>>>> Michael
>>>>>>>
>>>>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew<ar...@gmail.com>  <ar...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>
>>>>>>> Thanks Matthias.
>>>>>>> Sure, will correct it right away.
>>>>>>>
>>>>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax"<ma...@confluent.io>  <ma...@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Thanks for preparing this page!
>>>>>>>
>>>>>>> About terminology:
>>>>>>>
>>>>>>> You introduce the term "event time" -- but we should call this
>>>>>>> "stream
>>>>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>>>>> this
>>>>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>>>>
>>>>>>> Does this make sense to you?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>>>
>>>>>>> Thanks Ewen.
>>>>>>>
>>>>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>>>>
>>>>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>>>
>>>>>>>
>>>>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>>>>> public.
>>>>>>>
>>>>>>> --
>>>>>>> Arun Mathew
>>>>>>>
>>>>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava"<ew...@confluent.io>  <ew...@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>>      Arun,
>>>>>>>
>>>>>>>      I've given you permission to edit the wiki. Let me know if
>>>>>>> you run
>>>>>>>
>>>>>>> into any
>>>>>>>
>>>>>>>      issues.
>>>>>>>
>>>>>>>      -Ewen
>>>>>>>
>>>>>>>      On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew<amathew@yahoo-co rp.jp>  <am...@yahoo-corp.jp>
>>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>
>>>>>>>      > Thanks Michal. I don\u2019t have the access yet [arunmathew88].
>>>>>>> Should I
>>>>>>>
>>>>>>> be
>>>>>>>
>>>>>>>      > sending a separate mail for this?
>>>>>>>      >
>>>>>>>      > I thought one of the person following this thread would be
>>>>>>> able to
>>>>>>>
>>>>>>> give me
>>>>>>>
>>>>>>>      > access.
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > *From: *Michal Borowiecki<mi...@openbet.com>  <mi...@openbet.com>
>>>>>>>      > *Reply-To: *"dev@kafka.apache.org"  <de...@kafka.apache.org>  <de...@kafka.apache.org>  <de...@kafka.apache.org>
>>>>>>>      > *Date: *Friday, April 7, 2017 at 17:16
>>>>>>>      > *To: *"dev@kafka.apache.org"  <de...@kafka.apache.org>  <de...@kafka.apache.org>  <de...@kafka.apache.org>
>>>>>>>      > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>>>> semantics
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > Hi Arun,
>>>>>>>      >
>>>>>>>      > I was thinking along the same lines as you, listing the use
>>>>>>> cases
>>>>>>>
>>>>>>> on the
>>>>>>>
>>>>>>>      > wiki, but didn't find time to get around doing that yet.
>>>>>>>      > Don't mind if you do it if you have access now.
>>>>>>>      > I was thinking it would be nice if, once we have the use
>>>>>>> cases
>>>>>>>
>>>>>>> listed,
>>>>>>>
>>>>>>>      > people could use likes to up-vote the use cases similar to
>>>>>>> what
>>>>>>>
>>>>>>> they're
>>>>>>>
>>>>>>>      > working on.
>>>>>>>      >
>>>>>>>      > I should have a bit more time to action this in the next
>>>>>>> few days,
>>>>>>>
>>>>>>> but
>>>>>>>
>>>>>>>      > happy for you to do it if you can beat me to it ;-)
>>>>>>>      >
>>>>>>>      > Cheers,
>>>>>>>      > Michal
>>>>>>>      >
>>>>>>>      > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>>>      >
>>>>>>>      > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > Of course. I was thinking of a subpage where people can
>>>>>>>
>>>>>>> collaborate.
>>>>>>>
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > Will do as per Michael\u2019s suggestion.
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > Regards,
>>>>>>>      >
>>>>>>>      > Arun Mathew
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > On 4/7/17, 12:30, "Matthias J. Sax"<ma...@confluent.io>  <ma...@confluent.io>
>>>>>>> <
>>>>>>>
>>>>>>> matthias@confluent.io> wrote:
>>>>>>>
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >     Please share your Wiki-ID and a committer can give you
>>>>>>> write
>>>>>>>
>>>>>>> access.
>>>>>>>
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >     Btw: as you did not initiate the KIP, you should not
>>>>>>> change the
>>>>>>>
>>>>>>> KIP
>>>>>>>
>>>>>>>      >
>>>>>>>      >     without the permission of the original author -- in
>>>>>>> this case
>>>>>>>
>>>>>>> Michael.
>>>>>>>
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >     So you might also just share your thought over the
>>>>>>> mailing list
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>>      >
>>>>>>>      >     Michael can update the KIP page. Or, as an alternative,
>>>>>>> just
>>>>>>>
>>>>>>> create a
>>>>>>>
>>>>>>>      >
>>>>>>>      >     subpage for the KIP page.
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >     @Michael: WDYT?
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >     -Matthias
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>>>      >
>>>>>>>      >     > Hi Jay,
>>>>>>>      >
>>>>>>>      >     >           Thanks for the advise, I would like to list
>>>>>>> down
>>>>>>>
>>>>>>> the use cases as
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > per your suggestion. But it seems I don't have write
>>>>>>>
>>>>>>> permission to the
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > Apache Kafka Confluent Space. Whom shall I request
>>>>>>> for it?
>>>>>>>      >
>>>>>>>      >     >
>>>>>>>      >
>>>>>>>      >     > Regarding your last question. We are using a patch in
>>>>>>> our
>>>>>>>
>>>>>>> production system
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > which does exactly this.
>>>>>>>      >
>>>>>>>      >     > We window by the event time, but trigger punctuate in
>>>>>>>
>>>>>>> <punctuate interval>
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > duration of system time, in the absence of an event
>>>>>>> crossing
>>>>>>>
>>>>>>> the punctuate
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > event time.
>>>>>>>      >
>>>>>>>      >     >
>>>>>>>      >
>>>>>>>      >     > We are using Kafka Stream for our Audit Trail, where
>>>>>>> we need
>>>>>>>
>>>>>>> to output the
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > event counts on each topic on each cluster aggregated
>>>>>>> over a
>>>>>>>
>>>>>>> 1 minute
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > window. We have to use event time to be able to cross
>>>>>>> check
>>>>>>>
>>>>>>> the counts. But
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > we need to trigger punctuate [aggregate event pushes]
>>>>>>> by
>>>>>>>
>>>>>>> system time in the
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > absence of events. Otherwise the event counts for
>>>>>>> unexpired
>>>>>>>
>>>>>>> windows would
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > be 0 which is bad.
>>>>>>>      >
>>>>>>>      >     >
>>>>>>>      >
>>>>>>>      >     > "Maybe a hybrid solution works: I window by event
>>>>>>> time but
>>>>>>>
>>>>>>> trigger results
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > by system time for windows that have updated? Not
>>>>>>> really sure
>>>>>>>
>>>>>>> the details
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > of making that work. Does that work? Are there
>>>>>>> concrete
>>>>>>>
>>>>>>> examples where you
>>>>>>>
>>>>>>>      >
>>>>>>>      >     > actually want the current behavior?"
>>>>>>>      >
>>>>>>>      >     >
>>>>>>>      >
>>>>>>>      >     > --
>>>>>>>      >
>>>>>>>      >     > With Regards,
>>>>>>>      >
>>>>>>>      >     >
>>>>>>>      >
>>>>>>>      >     > Arun Mathew
>>>>>>>      >
>>>>>>>      >     > Yahoo! JAPAN Corporation
>>>>>>>      >
>>>>>>>      >     >
>>>>>>>      >
>>>>>>>      >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>>>>
>>>>>>> skyahead@gmail.com><sk...@gmail.com>  <sk...@gmail.com>  wrote:
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >
>>>>>>>      >
>>>>>>>      >     >> Hi Jay,
>>>>>>>      >
>>>>>>>      >     >>
>>>>>>>      >
>>>>>>>      >     >> The hybrid solution is exactly what I expect and
>>>>>>> need for
>>>>>>>
>>>>>>> our use cases
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> when dealing with telecom data.
>>>>>>>      >
>>>>>>>      >     >>
>>>>>>>      >
>>>>>>>      >     >> Thanks
>>>>>>>      >
>>>>>>>      >     >> Tianji
>>>>>>>      >
>>>>>>>      >     >>
>>>>>>>      >
>>>>>>>      >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>>>>
>>>>>>> jay@confluent.io><ja...@confluent.io>  <ja...@confluent.io>  wrote:
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>
>>>>>>>      >
>>>>>>>      >     >>> Hey guys,
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>> One thing I've always found super important for
>>>>>>> this kind
>>>>>>>
>>>>>>> of design work
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> is
>>>>>>>      >
>>>>>>>      >     >>> to do a really good job of cataloging the landscape
>>>>>>> of use
>>>>>>>
>>>>>>> cases and how
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> prevalent each one is. By that I mean not just
>>>>>>> listing lots
>>>>>>>
>>>>>>> of uses, but
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> also grouping them into categories that
>>>>>>> functionally need
>>>>>>>
>>>>>>> the same thing.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> In the absence of this it is very hard to reason
>>>>>>> about
>>>>>>>
>>>>>>> design proposals.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> From the proposals so far I think we have a lot of
>>>>>>>
>>>>>>> discussion around
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> possible apis, but less around what the user needs
>>>>>>> for
>>>>>>>
>>>>>>> different use
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> cases
>>>>>>>      >
>>>>>>>      >     >>> and how they would implement that using the api.
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>> Here is an example:
>>>>>>>      >
>>>>>>>      >     >>> You aggregate click and impression data for a
>>>>>>> reddit like
>>>>>>>
>>>>>>> site. Every ten
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> minutes you want to output a ranked list of the top
>>>>>>> 10
>>>>>>>
>>>>>>> articles ranked by
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> clicks/impressions for each geographical area. I
>>>>>>> want to be
>>>>>>>
>>>>>>> able run this
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> in steady state as well as rerun to regenerate
>>>>>>> results (or
>>>>>>>
>>>>>>> catch up if it
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> crashes).
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>> There are a couple of tricky things that seem to
>>>>>>> make this
>>>>>>>
>>>>>>> hard with
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> either
>>>>>>>      >
>>>>>>>      >     >>> of the options proposed:
>>>>>>>      >
>>>>>>>      >     >>> 1. If I emit this data using event time I have the
>>>>>>> problem
>>>>>>>
>>>>>>> described
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> where
>>>>>>>      >
>>>>>>>      >     >>> a geographical region with no new clicks or
>>>>>>> impressions
>>>>>>>
>>>>>>> will fail to
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> output
>>>>>>>      >
>>>>>>>      >     >>> results.
>>>>>>>      >
>>>>>>>      >     >>> 2. If I emit this data using system time I have the
>>>>>>> problem
>>>>>>>
>>>>>>> that when
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> reprocessing data my window may not be ten minutes
>>>>>>> but 10
>>>>>>>
>>>>>>> hours if my
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> processing is very fast so it dramatically changes
>>>>>>> the
>>>>>>>
>>>>>>> output.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>> Maybe a hybrid solution works: I window by event
>>>>>>> time but
>>>>>>>
>>>>>>> trigger results
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> by system time for windows that have updated? Not
>>>>>>> really
>>>>>>>
>>>>>>> sure the details
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> of making that work. Does that work? Are there
>>>>>>> concrete
>>>>>>>
>>>>>>> examples where
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> you
>>>>>>>      >
>>>>>>>      >     >>> actually want the current behavior?
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>> -Jay
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>>>>
>>>>>>> arunmathew88@gmail.com><ar...@gmail.com>  <ar...@gmail.com>
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> wrote:
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>>> Hi All,
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> Thanks for the KIP. We were also in need of a
>>>>>>> mechanism to
>>>>>>>
>>>>>>> trigger
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>> punctuate in the absence of events.
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> As I described in [
>>>>>>>      >
>>>>>>>      >     >>>>https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>>>      >
>>>>>>>      >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>>>      >
>>>>>>>      >     >>>> plugin.system.issuetabpanels:comment-
>>>>>>> tabpanel#comment-
>>>>>>>
>>>>>>> 15926036
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>> ],
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>>    - Our approached involved using the event time
>>>>>>> by
>>>>>>>
>>>>>>> default.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>    - The method to check if there is any punctuate
>>>>>>> ready
>>>>>>>
>>>>>>> in the
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>    PunctuationQueue is triggered via the any event
>>>>>>>
>>>>>>> received by the
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> stream
>>>>>>>      >
>>>>>>>      >     >>>>    tread, or at the polling intervals in the
>>>>>>> absence of
>>>>>>>
>>>>>>> any events.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>    - When we create Punctuate objects (which
>>>>>>> contains the
>>>>>>>
>>>>>>> next event
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> time
>>>>>>>      >
>>>>>>>      >     >>>>    for punctuation and interval), we also record
>>>>>>> the
>>>>>>>
>>>>>>> creation time
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> (system
>>>>>>>      >
>>>>>>>      >     >>>>    time).
>>>>>>>      >
>>>>>>>      >     >>>>    - While checking for maturity of Punctuate
>>>>>>> Schedule by
>>>>>>>      >
>>>>>>>      >     >> mayBePunctuate
>>>>>>>      >
>>>>>>>      >     >>>>    method, we also check if the system clock has
>>>>>>> elapsed
>>>>>>>
>>>>>>> the punctuate
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>    interval since the schedule creation time.
>>>>>>>      >
>>>>>>>      >     >>>>    - In the absence of any event, or in the
>>>>>>> absence of any
>>>>>>>
>>>>>>> event for
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> one
>>>>>>>      >
>>>>>>>      >     >>>>    topic in the partition group assigned to the
>>>>>>> stream
>>>>>>>
>>>>>>> task, the system
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>> time
>>>>>>>      >
>>>>>>>      >     >>>>    will elapse the interval and we trigger a
>>>>>>> punctuate
>>>>>>>
>>>>>>> using the
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> expected
>>>>>>>      >
>>>>>>>      >     >>>>    punctuation event time.
>>>>>>>      >
>>>>>>>      >     >>>>    - we then create the next punctuation schedule
>>>>>>> as
>>>>>>>
>>>>>>> punctuation event
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> time
>>>>>>>      >
>>>>>>>      >     >>>>    + punctuation interval, [again recording the
>>>>>>> system
>>>>>>>
>>>>>>> time of creation
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> of
>>>>>>>      >
>>>>>>>      >     >>>> the
>>>>>>>      >
>>>>>>>      >     >>>>    schedule].
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>>>> approach
>>>>>>>
>>>>>>> has pros and
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>> cons.
>>>>>>>      >
>>>>>>>      >     >>>> Pros
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>>    - Punctuates will happen in <punctuate
>>>>>>> interval> time
>>>>>>>
>>>>>>> duration at
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> max
>>>>>>>      >
>>>>>>>      >     >>> in
>>>>>>>      >
>>>>>>>      >     >>>>    terms of system time.
>>>>>>>      >
>>>>>>>      >     >>>>    - The semantics as a whole continues to revolve
>>>>>>> around
>>>>>>>
>>>>>>> event time.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>    - We can use the old data [old timestamps] to
>>>>>>> rerun any
>>>>>>>
>>>>>>> experiments
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> or
>>>>>>>      >
>>>>>>>      >     >>>>    tests.
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> Cons
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>>    - In case the  <punctuate interval> is not a
>>>>>>> time
>>>>>>>
>>>>>>> duration [say
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> logical
>>>>>>>      >
>>>>>>>      >     >>>>    time/event count], then the approach might not
>>>>>>> be
>>>>>>>
>>>>>>> meaningful.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>    - In case there is a case where we have to wait
>>>>>>> for an
>>>>>>>
>>>>>>> actual event
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> from
>>>>>>>      >
>>>>>>>      >     >>>>    a low event rate partition in the partition
>>>>>>> group, this
>>>>>>>
>>>>>>> approach
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> will
>>>>>>>      >
>>>>>>>      >     >>>> jump
>>>>>>>      >
>>>>>>>      >     >>>>    the gun.
>>>>>>>      >
>>>>>>>      >     >>>>    - in case the event processing cannot catch up
>>>>>>> with the
>>>>>>>
>>>>>>> event rate
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> and
>>>>>>>      >
>>>>>>>      >     >>>>    the expected timestamp events gets queued for
>>>>>>> long
>>>>>>>
>>>>>>> time, this
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> approach
>>>>>>>      >
>>>>>>>      >     >>>>    might jump the gun.
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> I believe the above approach and discussion goes
>>>>>>> close to
>>>>>>>
>>>>>>> the approach
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> A.
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> -----------
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> I like the idea of having an even count based
>>>>>>> punctuate.
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> -----------
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> I agree with the discussion around approach C,
>>>>>>> that we
>>>>>>>
>>>>>>> should provide
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> the
>>>>>>>      >
>>>>>>>      >     >>>> user with the option to choose system time or
>>>>>>> event time
>>>>>>>
>>>>>>> based
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> punctuates.
>>>>>>>      >
>>>>>>>      >     >>>> But I believe that the user predominantly wants to
>>>>>>> use
>>>>>>>
>>>>>>> event time while
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> not
>>>>>>>      >
>>>>>>>      >     >>>> missing out on regular punctuates due to event
>>>>>>> delays or
>>>>>>>
>>>>>>> event
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> absences.
>>>>>>>      >
>>>>>>>      >     >>>> Hence a complex punctuate option as Matthias
>>>>>>> mentioned
>>>>>>>
>>>>>>> (quoted below)
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> would
>>>>>>>      >
>>>>>>>      >     >>>> be most apt.
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> "- We might want to add "complex" schedules later
>>>>>>> on
>>>>>>>
>>>>>>> (like, punctuate
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> on
>>>>>>>      >
>>>>>>>      >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>>>> time
>>>>>>>
>>>>>>> whatever comes
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>> first)."
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> -----------
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> I think I read somewhere that Kafka Streams
>>>>>>> started with
>>>>>>>
>>>>>>> System Time as
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> the
>>>>>>>      >
>>>>>>>      >     >>>> punctuation standard, but was later changed to
>>>>>>> Event Time.
>>>>>>>
>>>>>>> I guess
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> there
>>>>>>>      >
>>>>>>>      >     >>>> would be some good reason behind it. As Kafka
>>>>>>> Streams want
>>>>>>>
>>>>>>> to evolve
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> more
>>>>>>>      >
>>>>>>>      >     >>>> on the Stream Processing front, I believe the
>>>>>>> emphasis on
>>>>>>>
>>>>>>> event time
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> would
>>>>>>>      >
>>>>>>>      >     >>>> remain quite strong.
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> With Regards,
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> Arun Mathew
>>>>>>>      >
>>>>>>>      >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>>>>
>>>>>>> tobecker@tivo.com><to...@tivo.com>  <to...@tivo.com>
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> wrote:
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>>> Yeah I like PuncutationType much better; I just
>>>>>>> threw
>>>>>>>
>>>>>>> Time out there
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>>>> still
>>>>>>>
>>>>>>> think it's
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> worth considering what this buys us over an
>>>>>>> additional
>>>>>>>
>>>>>>> callback. I
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> foresee a number of punctuate implementations
>>>>>>> following
>>>>>>>
>>>>>>> this pattern:
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> public void punctuate(PunctuationType type) {
>>>>>>>      >
>>>>>>>      >     >>>>>     switch (type) {
>>>>>>>      >
>>>>>>>      >     >>>>>         case EVENT_TIME:
>>>>>>>      >
>>>>>>>      >     >>>>>             methodA();
>>>>>>>      >
>>>>>>>      >     >>>>>             break;
>>>>>>>      >
>>>>>>>      >     >>>>>         case SYSTEM_TIME:
>>>>>>>      >
>>>>>>>      >     >>>>>             methodB();
>>>>>>>      >
>>>>>>>      >     >>>>>             break;
>>>>>>>      >
>>>>>>>      >     >>>>>     }
>>>>>>>      >
>>>>>>>      >     >>>>> }
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> I guess one advantage of this approach is we
>>>>>>> could add
>>>>>>>
>>>>>>> additional
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> punctuation types later in a backwards compatible
>>>>>>> way
>>>>>>>
>>>>>>> (like event
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> count
>>>>>>>      >
>>>>>>>      >     >>>>> as you mentioned).
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> -Tommy
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>>>> Sax wrote:
>>>>>>>      >
>>>>>>>      >     >>>>>> That sounds promising.
>>>>>>>      >
>>>>>>>      >     >>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>> I am just wondering if `Time` is the best name.
>>>>>>> Maybe we
>>>>>>>
>>>>>>> want to
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> add
>>>>>>>      >
>>>>>>>      >     >>>>>> other non-time based punctuations at some point
>>>>>>> later. I
>>>>>>>
>>>>>>> would
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>> suggest
>>>>>>>      >
>>>>>>>      >     >>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>> enum PunctuationType {
>>>>>>>      >
>>>>>>>      >     >>>>>>   EVENT_TIME,
>>>>>>>      >
>>>>>>>      >     >>>>>>   SYSTEM_TIME,
>>>>>>>      >
>>>>>>>      >     >>>>>> }
>>>>>>>      >
>>>>>>>      >     >>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>> or similar. Just to keep the door open -- it's
>>>>>>> easier to
>>>>>>>
>>>>>>> add new
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>> stuff
>>>>>>>      >
>>>>>>>      >     >>>>>> if the name is more generic.
>>>>>>>      >
>>>>>>>      >     >>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>> -Matthias
>>>>>>>      >
>>>>>>>      >     >>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> I agree that the framework providing and
>>>>>>> managing the
>>>>>>>
>>>>>>> notion of
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> stream
>>>>>>>      >
>>>>>>>      >     >>>>>>> time is valuable and not something we would
>>>>>>> want to
>>>>>>>
>>>>>>> delegate to
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> the
>>>>>>>      >
>>>>>>>      >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>>>> separate
>>>>>>>
>>>>>>> callback
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> (option
>>>>>>>      >
>>>>>>>      >     >>>>>>> C)
>>>>>>>      >
>>>>>>>      >     >>>>>>> is that messy (it could just be a default
>>>>>>> method with
>>>>>>>
>>>>>>> an empty
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> implementation), but if we wanted a single API
>>>>>>> to
>>>>>>>
>>>>>>> handle both
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> cases,
>>>>>>>      >
>>>>>>>      >     >>>>>>> how about something like the following?
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> enum Time {
>>>>>>>      >
>>>>>>>      >     >>>>>>>    STREAM,
>>>>>>>      >
>>>>>>>      >     >>>>>>>    CLOCK
>>>>>>>      >
>>>>>>>      >     >>>>>>> }
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> Then on ProcessorContext:
>>>>>>>      >
>>>>>>>      >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>>>> We could
>>>>>>>
>>>>>>> allow
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> this
>>>>>>>      >
>>>>>>>      >     >>>>>>> to
>>>>>>>      >
>>>>>>>      >     >>>>>>> be called once for each value of time to mix
>>>>>>>
>>>>>>> approaches.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> Then the Processor API becomes:
>>>>>>>      >
>>>>>>>      >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>>>>
>>>>>>> schedule resulted
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> in
>>>>>>>      >
>>>>>>>      >     >>>>>>> this call.
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> Thoughts?
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>>>> Sax
>>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> I was thinking about the four options you
>>>>>>> proposed in
>>>>>>>
>>>>>>> more
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> details
>>>>>>>      >
>>>>>>>      >     >>>>>>>> and
>>>>>>>      >
>>>>>>>      >     >>>>>>>> this are my thoughts:
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> (A) You argue, that users can still
>>>>>>> "punctuate" on
>>>>>>>
>>>>>>> event-time
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> via
>>>>>>>      >
>>>>>>>      >     >>>>>>>> process(), but I am not sure if this is
>>>>>>> possible.
>>>>>>>
>>>>>>> Note, that
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> users
>>>>>>>      >
>>>>>>>      >     >>>>>>>> only
>>>>>>>      >
>>>>>>>      >     >>>>>>>> get record timestamps via context.timestamp().
>>>>>>> Thus,
>>>>>>>
>>>>>>> users
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> would
>>>>>>>      >
>>>>>>>      >     >>>>>>>> need
>>>>>>>      >
>>>>>>>      >     >>>>>>>> to
>>>>>>>      >
>>>>>>>      >     >>>>>>>> track the time progress per partition (based
>>>>>>> on the
>>>>>>>
>>>>>>> partitions
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> they
>>>>>>>      >
>>>>>>>      >     >>>>>>>> obverse via context.partition(). (This alone
>>>>>>> puts a
>>>>>>>
>>>>>>> huge burden
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> on
>>>>>>>      >
>>>>>>>      >     >>>>>>>> the
>>>>>>>      >
>>>>>>>      >     >>>>>>>> user by itself.) However, users are not
>>>>>>> notified at
>>>>>>>
>>>>>>> startup
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> what
>>>>>>>      >
>>>>>>>      >     >>>>>>>> partitions are assigned, and user are not
>>>>>>> notified
>>>>>>>
>>>>>>> when
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> partitions
>>>>>>>      >
>>>>>>>      >     >>>>>>>> get
>>>>>>>      >
>>>>>>>      >     >>>>>>>> revoked. Because this information is not
>>>>>>> available,
>>>>>>>
>>>>>>> it's not
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> possible
>>>>>>>      >
>>>>>>>      >     >>>>>>>> to
>>>>>>>      >
>>>>>>>      >     >>>>>>>> "manually advance" stream-time, and thus
>>>>>>> event-time
>>>>>>>
>>>>>>> punctuation
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> within
>>>>>>>      >
>>>>>>>      >     >>>>>>>> process() seems not to be possible -- or do
>>>>>>> you see a
>>>>>>>
>>>>>>> way to
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> get
>>>>>>>      >
>>>>>>>      >     >>>>>>>> it
>>>>>>>      >
>>>>>>>      >     >>>>>>>> done? And even if, it might still be too
>>>>>>> clumsy to
>>>>>>>
>>>>>>> use.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> (B) This does not allow to mix both
>>>>>>> approaches, thus
>>>>>>>
>>>>>>> limiting
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> what
>>>>>>>      >
>>>>>>>      >     >>>>>>>> users
>>>>>>>      >
>>>>>>>      >     >>>>>>>> can do.
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> (C) This should give all flexibility we need.
>>>>>>> However,
>>>>>>>
>>>>>>> just
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> adding
>>>>>>>      >
>>>>>>>      >     >>>>>>>> one
>>>>>>>      >
>>>>>>>      >     >>>>>>>> more method seems to be a solution that is too
>>>>>>> simple
>>>>>>>
>>>>>>> (cf my
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> comments
>>>>>>>      >
>>>>>>>      >     >>>>>>>> below).
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>>>> sure how
>>>>>>>
>>>>>>> a user
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> could
>>>>>>>      >
>>>>>>>      >     >>>>>>>> enable system-time and event-time punctuation
>>>>>>> in
>>>>>>>
>>>>>>> parallel.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> Overall options (C) seems to be the most
>>>>>>> promising
>>>>>>>
>>>>>>> approach to
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> me.
>>>>>>>      >
>>>>>>>      >     >>>>>>>> Because I also favor a clean API, we might
>>>>>>> keep
>>>>>>>
>>>>>>> current
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> punctuate()
>>>>>>>      >
>>>>>>>      >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>>>> at some
>>>>>>>
>>>>>>> later
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> point
>>>>>>>      >
>>>>>>>      >     >>>>>>>> when
>>>>>>>      >
>>>>>>>      >     >>>>>>>> people use the "new punctuate API".
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> Couple of follow up questions:
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> - I am wondering, if we should have two
>>>>>>> callback
>>>>>>>
>>>>>>> methods or
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> just
>>>>>>>      >
>>>>>>>      >     >>>>>>>> one
>>>>>>>      >
>>>>>>>      >     >>>>>>>> (ie, a unified for system and event time
>>>>>>> punctuation
>>>>>>>
>>>>>>> or one for
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> each?).
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> - If we have one, how can the user figure out,
>>>>>>> which
>>>>>>>
>>>>>>> condition
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> did
>>>>>>>      >
>>>>>>>      >     >>>>>>>> trigger?
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> - How would the API look like, for registering
>>>>>>>
>>>>>>> different
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> punctuate
>>>>>>>      >
>>>>>>>      >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> - We might want to add "complex" schedules
>>>>>>> later on
>>>>>>>
>>>>>>> (like,
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> punctuate
>>>>>>>      >
>>>>>>>      >     >>>>>>>> on
>>>>>>>      >
>>>>>>>      >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>>>> system-time
>>>>>>>
>>>>>>> whatever
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> comes
>>>>>>>      >
>>>>>>>      >     >>>>>>>> first). I don't say we should add this right
>>>>>>> away, but
>>>>>>>
>>>>>>> we might
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> want
>>>>>>>      >
>>>>>>>      >     >>>>>>>> to
>>>>>>>      >
>>>>>>>      >     >>>>>>>> define the API in a way, that it allows
>>>>>>> extensions
>>>>>>>
>>>>>>> like this
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> later
>>>>>>>      >
>>>>>>>      >     >>>>>>>> on,
>>>>>>>      >
>>>>>>>      >     >>>>>>>> without redesigning the API (ie, the API
>>>>>>> should be
>>>>>>>
>>>>>>> designed
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> extensible)
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> - Did you ever consider count-based
>>>>>>> punctuation?
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> I understand, that you would like to solve a
>>>>>>> simple
>>>>>>>
>>>>>>> problem,
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> but
>>>>>>>      >
>>>>>>>      >     >>>>>>>> we
>>>>>>>      >
>>>>>>>      >     >>>>>>>> learned from the past, that just "adding some
>>>>>>> API"
>>>>>>>
>>>>>>> quickly
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> leads
>>>>>>>      >
>>>>>>>      >     >>>>>>>> to a
>>>>>>>      >
>>>>>>>      >     >>>>>>>> not very well defined API that needs time
>>>>>>> consuming
>>>>>>>
>>>>>>> clean up
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> later on
>>>>>>>      >
>>>>>>>      >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>>>> holistic
>>>>>>>      >
>>>>>>>      >     >>>>>>>> punctuation
>>>>>>>      >
>>>>>>>      >     >>>>>>>> KIP
>>>>>>>      >
>>>>>>>      >     >>>>>>>> with this from the beginning on to avoid later
>>>>>>> painful
>>>>>>>      >
>>>>>>>      >     >> redesign.
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> -Matthias
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> Thanks Thomas,
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> I'm also wary of changing the existing
>>>>>>> semantics of
>>>>>>>      >
>>>>>>>      >     >> punctuate,
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> for
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> backward compatibility reasons, although I
>>>>>>> like the
>>>>>>>      >
>>>>>>>      >     >> conceptual
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> simplicity of that option.
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>>>> a way,
>>>>>>>
>>>>>>> uglier.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> I
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> added
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> this to the KIP now as option (C).
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>>>> more
>>>>>>>
>>>>>>> flexible,
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> as
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> it
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> allows
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> you to return any value, you're not limited
>>>>>>> to event
>>>>>>>
>>>>>>> time or
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> system
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> time
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> (although I don't see an actual use case
>>>>>>> where you
>>>>>>>
>>>>>>> might need
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> anything
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> else then those two). Hence I also proposed
>>>>>>> the
>>>>>>>
>>>>>>> option to
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> allow
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> users
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>>>> for
>>>>>>>
>>>>>>> them given
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> the
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> presence or absence of messages, much like
>>>>>>> they can
>>>>>>>
>>>>>>> decide
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> what
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> msg
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> time
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>>>> What do
>>>>>>>
>>>>>>> you
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> think
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> about
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> that? This is probably most flexible but also
>>>>>>> most
>>>>>>>      >
>>>>>>>      >     >> complicated.
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> All comments appreciated.
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> Cheers,
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> Michal
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> Although I fully agree we need a way to
>>>>>>> trigger
>>>>>>>
>>>>>>> periodic
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> processing
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> that is independent from whether and when
>>>>>>> messages
>>>>>>>
>>>>>>> arrive,
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> I'm
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> not sure
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> I like the idea of changing the existing
>>>>>>> semantics
>>>>>>>
>>>>>>> across
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> the
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> board.
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> What if we added an additional callback to
>>>>>>> Processor
>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> can
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> be
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>>>> always
>>>>>>>
>>>>>>> called at
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> fixed, wall
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>>>> have to
>>>>>>>
>>>>>>> give
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> up
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> the
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> notion
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> of stream time to be able to do periodic
>>>>>>> processing.
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>>>> Borowiecki
>>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> Hi all,
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>>>>
>>>>>>> punctuate
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> semantics
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>>>>
>>>>>>> confluence/display/KAFKA/KIP-<https://cwiki.apache.org/ confluence/display/KAFKA/KIP->  <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>>>
>>>>>>>      >
>>>>>>>      >     ><https://cwiki.apache.org/confluence/display/KAFKA/KI P->  <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>>>>
>>>>>>> 138%
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> 3A+C
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> hange+
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> punctuate+semantics>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> .
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> Appreciating there can be different views
>>>>>>> on
>>>>>>>
>>>>>>> system-time
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> vs
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> event-
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> time
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>>>> case and
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> importance of
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>>>> I've
>>>>>>>
>>>>>>> left it
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> quite
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> open
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> and
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>>>>
>>>>>>> progresses.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> Thanks,
>>>>>>>      >
>>>>>>>      >     >>>>>>>>>>> Michal
>>>>>>>      >
>>>>>>>      >     >>>>>>> --
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>     Tommy Becker
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>     Senior Software Engineer
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>     tivo.com
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> ________________________________
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> This email and any attachments may contain
>>>>>>> confidential
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> privileged material for the sole use of the
>>>>>>> intended
>>>>>>>
>>>>>>> recipient.
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> Any
>>>>>>>      >
>>>>>>>      >     >>>>>>> review, copying, or distribution of this email
>>>>>>> (or any
>>>>>>>      >
>>>>>>>      >     >> attachments)
>>>>>>>      >
>>>>>>>      >     >>>>>>> by others is prohibited. If you are not the
>>>>>>> intended
>>>>>>>
>>>>>>> recipient,
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> please contact the sender immediately and
>>>>>>> permanently
>>>>>>>
>>>>>>> delete this
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> email and any attachments. No employee or agent
>>>>>>> of TiVo
>>>>>>>
>>>>>>> Inc. is
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> authorized to conclude any binding agreement on
>>>>>>> behalf
>>>>>>>
>>>>>>> of TiVo
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> Inc.
>>>>>>>      >
>>>>>>>      >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>>>> only be
>>>>>>>
>>>>>>> made by a
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>>> signed written agreement.
>>>>>>>      >
>>>>>>>      >     >>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> --
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>     Tommy Becker
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>     Senior Software Engineer
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>     tivo.com
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> ________________________________
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> This email and any attachments may contain
>>>>>>> confidential
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> privileged
>>>>>>>      >
>>>>>>>      >     >>>>> material for the sole use of the intended
>>>>>>> recipient. Any
>>>>>>>
>>>>>>> review,
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>> copying,
>>>>>>>      >
>>>>>>>      >     >>>>> or distribution of this email (or any
>>>>>>> attachments) by
>>>>>>>
>>>>>>> others is
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>> prohibited.
>>>>>>>      >
>>>>>>>      >     >>>>> If you are not the intended recipient, please
>>>>>>> contact the
>>>>>>>
>>>>>>> sender
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> immediately and permanently delete this email and
>>>>>>> any
>>>>>>>
>>>>>>> attachments. No
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>>>> conclude
>>>>>>>
>>>>>>> any binding
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>>>> Binding
>>>>>>>
>>>>>>> agreements with
>>>>>>>
>>>>>>>      >
>>>>>>>      >     >> TiVo
>>>>>>>      >
>>>>>>>      >     >>>>> Inc. may only be made by a signed written
>>>>>>> agreement.
>>>>>>>      >
>>>>>>>      >     >>>>>
>>>>>>>      >
>>>>>>>      >     >>>>
>>>>>>>      >
>>>>>>>      >     >>>
>>>>>>>      >
>>>>>>>      >     >>
>>>>>>>      >
>>>>>>>      >     >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > --
>>>>>>>      >
>>>>>>>      ><http://www.openbet.com/>  <http://www.openbet.com/>
>>>>>>>
>>>>>>>      >
>>>>>>>      > *Michal Borowiecki*
>>>>>>>      >
>>>>>>>      > *Senior Software Engineer L4*
>>>>>>>      >
>>>>>>>      > *T: *
>>>>>>>      >
>>>>>>>      > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>>>>      >
>>>>>>>      > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > *E: *
>>>>>>>      >
>>>>>>>      >michal.borowiecki@openbet.com
>>>>>>>      >
>>>>>>>      > *W: *
>>>>>>>      >
>>>>>>>      >www.openbet.com
>>>>>>>      >
>>>>>>>      > *OpenBet Ltd*
>>>>>>>      >
>>>>>>>      > Chiswick Park Building 9
>>>>>>>      >
>>>>>>>      > 566 Chiswick High Rd
>>>>>>>      >
>>>>>>>      > London
>>>>>>>      >
>>>>>>>      > W4 5XT
>>>>>>>      >
>>>>>>>      > UK
>>>>>>>      >
>>>>>>>      ><https://www.openbet.com/email_promo>  <https://www.openbet.com/email_promo>
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>      > This message is confidential and intended only for the
>>>>>>> addressee.
>>>>>>>
>>>>>>> If you
>>>>>>>
>>>>>>>      > have received this message in error, please immediately
>>>>>>> notify the
>>>>>>>      >postmaster@openbet.com  and delete it from your system as
>>>>>>> well as
>>>>>>>
>>>>>>> any
>>>>>>>
>>>>>>>      > copies. The content of e-mails as well as traffic data may
>>>>>>> be
>>>>>>>
>>>>>>> monitored by
>>>>>>>
>>>>>>>      > OpenBet for employment and security purposes. To protect
>>>>>>> the
>>>>>>>
>>>>>>> environment
>>>>>>>
>>>>>>>      > please do not print this e-mail unless necessary. OpenBet
>>>>>>> Ltd.
>>>>>>>
>>>>>>> Registered
>>>>>>>
>>>>>>>      > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>>>> London,
>>>>>>>
>>>>>>> W4 5XT,
>>>>>>>
>>>>>>>      > United Kingdom. A company registered in England and Wales.
>>>>>>>
>>>>>>> Registered no.
>>>>>>>
>>>>>>>      > 3134634. VAT no. GB927523612
>>>>>>>      >
>>>>>>>      >
>>>>>>>      >
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>>
>>>>>>>
>>>>>>>      Tommy Becker
>>>>>>>
>>>>>>>      Senior Software Engineer
>>>>>>>
>>>>>>>      O +1 919.460.4747 <%28919%29%20460-4747>
>>>>>>>
>>>>>>>
>>>>>>>      tivo.com
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> ________________________________
>>>>>>>
>>>>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> <http://www.openbet.com/>  Michal Borowiecki
>>>>>>> Senior Software Engineer L4
>>>>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>>>>
>>>>>>>
>>>>>>>
>>>> -- 
>>>> Signature
>>>> <http://www.openbet.com/>  	Michal Borowiecki
>>>> Senior Software Engineer L4
>>>> 	T: 	+44 208 742 1600
>>>>
>>>> 	
>>>> 	+44 203 249 8448
>>>>
>>>> 	
>>>> 	
>>>> 	E: 	michal.borowiecki@openbet.com
>>>> 	W: 	www.openbet.com  <http://www.openbet.com/>
>>>>
>>>> 	
>>>> 	OpenBet Ltd
>>>>
>>>> 	Chiswick Park Building 9
>>>>
>>>> 	566 Chiswick High Rd
>>>>
>>>> 	London
>>>>
>>>> 	W4 5XT
>>>>
>>>> 	UK
>>>>
>>>> 	
>>>> <https://www.openbet.com/email_promo>
>>>>
>>>> This message is confidential and intended only for the addressee. If
>>>> you have received this message in error, please immediately notify the
>>>> postmaster@openbet.com  <ma...@openbet.com>  and delete it
>>>> from your system as well as any copies. The content of e-mails as well
>>>> as traffic data may be monitored by OpenBet for employment and
>>>> security purposes. To protect the environment please do not print this
>>>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>>> company registered in England and Wales. Registered no. 3134634. VAT
>>>> no. GB927523612
>>>>
>>> -- 
>>> Signature
>>> <http://www.openbet.com/>  	Michal Borowiecki
>>> Senior Software Engineer L4
>>> 	T: 	+44 208 742 1600
>>>
>>> 	
>>> 	+44 203 249 8448
>>>
>>> 	
>>> 	
>>> 	E: 	michal.borowiecki@openbet.com
>>> 	W: 	www.openbet.com  <http://www.openbet.com/>
>>>
>>> 	
>>> 	OpenBet Ltd
>>>
>>> 	Chiswick Park Building 9
>>>
>>> 	566 Chiswick High Rd
>>>
>>> 	London
>>>
>>> 	W4 5XT
>>>
>>> 	UK
>>>
>>> 	
>>> <https://www.openbet.com/email_promo>
>>>
>>> This message is confidential and intended only for the addressee. If you
>>> have received this message in error, please immediately notify the
>>> postmaster@openbet.com  <ma...@openbet.com>  and delete it
>>> from your system as well as any copies. The content of e-mails as well
>>> as traffic data may be monitored by OpenBet for employment and security
>>> purposes. To protect the environment please do not print this e-mail
>>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>> registered in England and Wales. Registered no. 3134634. VAT no.
>>> GB927523612
>>>
>
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
>
> 	
> 	+44 203 249 8448
>
> 	
> 	
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
>
> 	
> 	OpenBet Ltd
>
> 	Chiswick Park Building 9
>
> 	566 Chiswick High Rd
>
> 	London
>
> 	W4 5XT
>
> 	UK
>
> 	
> <https://www.openbet.com/email_promo>
>
> This message is confidential and intended only for the addressee. If 
> you have received this message in error, please immediately notify the 
> postmaster@openbet.com <ma...@openbet.com> and delete it 
> from your system as well as any copies. The content of e-mails as well 
> as traffic data may be monitored by OpenBet for employment and 
> security purposes. To protect the environment please do not print this 
> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park 
> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A 
> company registered in England and Wales. Registered no. 3134634. VAT 
> no. GB927523612
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Thanks!

Would a dashboard need perfect regularity? Wouldn't an upper bound suffice?

Unless too frequent messages on replay could overpower it?


I thought about cron-type things, but aren't they better triggered by an 
external scheduler (they're more flexible anyway), which then feeds 
"commands" into the topology?

Just my 2c.

Cheers,

Michal


On 24/04/17 17:57, Matthias J. Sax wrote:
> A simple example would be some dashboard app, that needs to get
> "current" status in regular time intervals (ie, and real-time app).
>
> Or something like a "scheduler" -- think "cron job" application.
>
>
> -Matthias
>
> On 4/24/17 2:23 AM, Michal Borowiecki wrote:
>> Hi Matthias,
>>
>> I agree it's difficult to reason about the hybrid approach, I certainly
>> found it hard and I'm totally on board with the mantra.
>>
>> I'd be happy to limit the scope of this KIP to add system-time
>> punctuation semantics (in addition to existing stream-time semantics)
>> and leave more complex schemes for users to implement on top of that.
>>
>> Further additional PunctuationTypes, could then be added by future KIPs,
>> possibly including the hybrid approach once it has been given more thought.
>>
>>> There are real-time applications, that want to get
>>> callbacks in regular system-time intervals (completely independent from
>>> stream-time).
>> Can you please describe what they are, so that I can put them on the
>> wiki for later reference?
>>
>> Thanks,
>>
>> Michal
>>
>>
>> On 23/04/17 21:27, Matthias J. Sax wrote:
>>> Hi,
>>>
>>> I do like Damian's API proposal about the punctuation callback function.
>>>
>>> I also did reread the KIP and thought about the semantics we want to
>>> provide.
>>>
>>>> Given the above, I don't see a reason any more for a separate system-time based punctuation.
>>> I disagree here. There are real-time applications, that want to get
>>> callbacks in regular system-time intervals (completely independent from
>>> stream-time). Thus we should allow this -- if we really follow the
>>> "hybrid" approach, this could be configured with stream-time interval
>>> infinite and delay whatever system-time punctuation interval you want to
>>> have. However, I would like to add a proper API for this and do this
>>> configuration under the hood (that would allow one implementation within
>>> all kind of branching for different cases).
>>>
>>> Thus, we definitely should have PunctutionType#StreamTime and
>>> #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
>>> a fan of your latest API proposal.
>>>
>>>
>>> About the hybrid approach in general. On the one hand I like it, on the
>>> other hand, it seems to be rather (1) complicated (not necessarily from
>>> an implementation point of view, but for people to understand it) and
>>> (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
>>>
>>>> It may appear complicated at first but I do think these semantics will
>>>> still be more understandable to users than having 2 separate punctuation
>>>> schedules/callbacks with different PunctuationTypes.
>>> This statement only holds if you apply strong assumptions that I don't
>>> believe hold in general -- see (2) for details -- and I think it is
>>> harder than you assume to reason about the hybrid approach in general.
>>> IMHO, the hybrid approach is a "false friend" that seems to be easy to
>>> reason about...
>>>
>>>
>>> (1) Streams always embraced "easy to use" and we should really be
>>> careful to keep it this way. On the other hand, as we are talking about
>>> changes to PAPI, it won't affect DSL users (DSL does not use punctuation
>>> at all at the moment), and thus, the "easy to use" mantra might not be
>>> affected, while it will allow advanced users to express more complex stuff.
>>>
>>> I like the mantra: "make simple thing easy and complex things possible".
>>>
>>> (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
>>> implicit assumption that even-time progresses at the same "speed" as
>>> system-time during regular processing. This implies the assumption that
>>> a slower progress in stream-time indicates the absence of input events
>>> (and that later arriving input events will have a larger event-time with
>>> high probability). Even if this might be true for some use cases, I
>>> doubt it holds in general. Assume that you get a spike in traffic and
>>> for some reason stream-time does advance slowly because you have more
>>> records to process. This might trigger a system-time based punctuation
>>> call even if this seems not to be intended. I strongly believe that it
>>> is not easy to reason about the semantics of the hybrid approach (even
>>> if the intentional semantics would be super useful -- but I doubt that
>>> we get want we ask for).
>>>
>>> Thus, I also believe that one might need different "configuration"
>>> values for the hybrid approach if you run the same code for different
>>> scenarios: regular processing, re-processing, catching up scenario. And
>>> as the term "configuration" implies, we might be better off to not mix
>>> configuration with business logic that is expressed via code.
>>>
>>>
>>> One more comment: I also don't think that the hybrid approach is
>>> deterministic as claimed in the use-case subpage. I understand the
>>> reasoning and agree, that it is deterministic if certain assumptions
>>> hold -- compare above -- and if configured correctly. But strictly
>>> speaking it's not because there is a dependency on system-time (and
>>> IMHO, if system-time is involved it cannot be deterministic by definition).
>>>
>>>
>>>> I see how in theory this could be implemented on top of the 2 punctuate
>>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>>> the other system-time based) but it would be a much more complicated
>>>> scheme and I don't want to suggest that.
>>> I agree that expressing the intended hybrid semantics is harder if we
>>> offer only #StreamTime and #SystemTime punctuation. However, I also
>>> believe that the hybrid approach is a "false friend" with regard to
>>> reasoning about the semantics (it indicates that it more easy as it is
>>> in reality). Therefore, we might be better off to not offer the hybrid
>>> approach and make it clear to a developed, that it is hard to mix
>>> #StreamTime and #SystemTime in a semantically sound way.
>>>
>>>
>>> Looking forward to your feedback. :)
>>>
>>> -Matthias
>>>
>>>
>>>
>>>
>>> On 4/22/17 11:43 AM, Michal Borowiecki wrote:
>>>> Hi all,
>>>>
>>>> Looking for feedback on the functional interface approach Damian
>>>> proposed. What do people think?
>>>>
>>>> Further on the semantics of triggering punctuate though:
>>>>
>>>> I ran through the 2 use cases that Arun had kindly put on the wiki
>>>> (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
>>>> in my head and on a whiteboard and I can't find a better solution than
>>>> the "hybrid" approach he had proposed.
>>>>
>>>> I see how in theory this could be implemented on top of the 2 punctuate
>>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>>> the other system-time based) but it would be a much more complicated
>>>> scheme and I don't want to suggest that.
>>>>
>>>> However, to add to the hybrid algorithm proposed, I suggest one
>>>> parameter to that: a tolerance period, expressed in milliseconds
>>>> system-time, after which the punctuation will be invoked in case the
>>>> stream-time advance hasn't triggered it within the requested interval
>>>> since the last invocation of punctuate (as recorded in system-time)
>>>>
>>>> This would allow a user-defined tolerance for late arriving events. The
>>>> trade off would be left for the user to decide: regular punctuation in
>>>> the case of absence of events vs allowing for records arriving late or
>>>> some build-up due to processing not catching up with the event rate.
>>>> In the one extreme, this tolerance could be set to infinity, turning
>>>> hybrid into simply stream-time based punctuate, like we have now. In the
>>>> other extreme, the tolerance could be set to 0, resulting in a
>>>> system-time upper bound on the effective punctuation interval.
>>>>
>>>> Given the above, I don't see a reason any more for a separate
>>>> system-time based punctuation. The "hybrid" approach with 0ms tolerance
>>>> would under normal operation trigger at regular intervals wrt the
>>>> system-time, except in cases of re-play/catch-up, where the stream time
>>>> advances faster than system time. In these cases punctuate would happen
>>>> more often than the specified interval wrt system time. However, the
>>>> use-cases that need system-time punctuations (that I've seen at least)
>>>> really only have a need for an upper bound on punctuation delay but
>>>> don't need a lower bound.
>>>>
>>>> To that effect I'd propose the api to be as follows, on ProcessorContext:
>>>>
>>>> schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
>>>>
>>>> schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
>>>>
>>>> Punctuation is triggered when either:
>>>> - the stream time advances past the (stream time of the previous
>>>> punctuation) + interval;
>>>> - or (iff the toleranceInterval is set) when the system time advances
>>>> past the (system time of the previous punctuation) + interval +
>>>> toleranceInterval
>>>>
>>>> In either case:
>>>> - we trigger punctuate passing as the argument the stream time at which
>>>> the current punctuation was meant to happen
>>>> - next punctuate is scheduled at (stream time at which the current
>>>> punctuation was meant to happen) + interval
>>>>
>>>> It may appear complicated at first but I do think these semantics will
>>>> still be more understandable to users than having 2 separate punctuation
>>>> schedules/callbacks with different PunctuationTypes.
>>>>
>>>>
>>>>
>>>> PS. Having re-read this, maybe the following alternative would be easier
>>>> to understand (WDYT?):
>>>>
>>>> schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
>>>>
>>>> schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
>>>>
>>>> Punctuation is triggered when either:
>>>> - the stream time advances past the (stream time of the previous
>>>> punctuation) + streamTimeInterval;
>>>> - or (iff systemTimeUpperBound is set) when the system time advances
>>>> past the (system time of the previous punctuation) + systemTimeUpperBound
>>>>
>>>> Awaiting comments.
>>>>
>>>> Thanks,
>>>> Michal
>>>>
>>>> On 21/04/17 16:56, Michal Borowiecki wrote:
>>>>> Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>>>>> in favour of something that doesn't return a record. Not a problem though.
>>>>>
>>>>>
>>>>> On 21/04/17 16:32, Damian Guy wrote:
>>>>>> Thanks Michal,
>>>>>> I agree Transformer.punctuate should also be void, but we can deprecate
>>>>>> that too in favor of the new interface.
>>>>>>
>>>>>> Thanks for the javadoc PR!
>>>>>>
>>>>>> Cheers,
>>>>>> Damian
>>>>>>
>>>>>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>
>>>>>>> Yes, that looks better to me.
>>>>>>>
>>>>>>> Note that punctuate on Transformer is currently returning a record, but I
>>>>>>> think it's ok to have all output records be sent via
>>>>>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>>>>>> multiple records from one invocation of punctuate.
>>>>>>>
>>>>>>> This way it's consistent between Processor and Transformer.
>>>>>>>
>>>>>>>
>>>>>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>>>>>> there:
>>>>>>>
>>>>>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>>>>>
>>>>>>> and PR: https://github.com/apache/kafka/pull/2884
>>>>>>>
>>>>>>> Cheers,
>>>>>>>
>>>>>>> Michal
>>>>>>> On 20/04/17 18:55, Damian Guy wrote:
>>>>>>>
>>>>>>> Hi Michal,
>>>>>>>
>>>>>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>>>>>> the API.
>>>>>>> 1. deprecate the punctuate method on Processor
>>>>>>> 2. create a new Functional Interface just for Punctuation, something like:
>>>>>>> interface Punctuator {
>>>>>>>      void punctuate(long timestamp)
>>>>>>> }
>>>>>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>>>>>> interval, PunctuationType type, Punctuator callback)
>>>>>>> 4. deprecate the existing schedule function
>>>>>>>
>>>>>>> Thoughts?
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Damian
>>>>>>>
>>>>>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>>
>>>>>>>> Hi Thomas,
>>>>>>>>
>>>>>>>> I would say our use cases fall in the same category as yours.
>>>>>>>>
>>>>>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>>>>>
>>>>>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>>>>>> of design:
>>>>>>>>
>>>>>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>>>>>
>>>>>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>>>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>>>>>
>>>>>>>> Since the output field X is derived in some non-trivial way, we don't
>>>>>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>>>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>>>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>>>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>>>>>> another input field Z).
>>>>>>>>
>>>>>>>> So we have kv stores with the records and an additional kv store with
>>>>>>>> timestamp->id mapping which act like an index where we periodically do a
>>>>>>>> ranged query.
>>>>>>>>
>>>>>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>>>>>> work when there were no regular msgs on the input topic.
>>>>>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>>>>>> outside we created a "ticker" that produced msgs once per second onto
>>>>>>>> another topic and fed it into the same topology to trigger punctuate.
>>>>>>>> This didn't work either, which was much more surprising to us at the
>>>>>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>>>>>> *all* input partitions receive messages regularly.
>>>>>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>>>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>>>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>>>>>> is consumed by the second topology and is its only input topic. There's a
>>>>>>>> transformer on that topic which populates and updates the time-based
>>>>>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>>>>>> elapsed, the record id is sent to the main transformer, which
>>>>>>>> updates/deletes the record from the main kv store and forwards the
>>>>>>>> transformed record to the output topic.
>>>>>>>>
>>>>>>>> To me this setup feels horrendously complicated for what it does.
>>>>>>>>
>>>>>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>>>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>>>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>>>>>> However, the ticks don't feel so hacky when you realise they give you
>>>>>>>> some hypothetical benefits in predictability. You can reprocess the
>>>>>>>> messages in a reproducible manner, since the topologies use event-time,
>>>>>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>>>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>>>>>> reprocessing).
>>>>>>>> To make that work though, we would have to have the stream time advance
>>>>>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>>>>>> presence of messages on the other input topic.
>>>>>>>>
>>>>>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>>>>>> and the hybrid would work to simplify this a lot.
>>>>>>>>
>>>>>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>>>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>>>>>> have to go with punctuate there too.
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Michal
>>>>>>>>
>>>>>>>>
>>>>>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>>>>>
>>>>>>>> Here's an example that we currently have.  We have a streams processor
>>>>>>>> that does a transform from one topic into another. One of the fields in
>>>>>>>> the source topic record is an expiration time, and one of the functions
>>>>>>>> of the processor is to ensure that expired records get deleted promptly
>>>>>>>> after that time passes (typically days or weeks after the message was
>>>>>>>> originally produced). To do that, the processor keeps a state store of
>>>>>>>> keys and expiration times, iterates that store in punctuate(), and
>>>>>>>> emits delete (null) records for expired items. This needs to happen at
>>>>>>>> some minimum interval regardless of the incoming message rate of the
>>>>>>>> source topic.
>>>>>>>>
>>>>>>>> In this scenario, the expiration of records is the primary function of
>>>>>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>>>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>>>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>>>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>>>>>> range of use-cases.
>>>>>>>>
>>>>>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>>>>>
>>>>>>>> I apologize for the longer email below.  To my defense, it started
>>>>>>>> out much
>>>>>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>>>>>> devil's
>>>>>>>> advocate for a number of arguments brought forth in order to help
>>>>>>>> improve
>>>>>>>> this KIP -- I am not implying I necessarily disagree with the
>>>>>>>> arguments.
>>>>>>>>
>>>>>>>> That aside, here are some further thoughts.
>>>>>>>>
>>>>>>>> First, there are (at least?) two categories for actions/behavior you
>>>>>>>> invoke
>>>>>>>> via punctuate():
>>>>>>>>
>>>>>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>>>>>> to
>>>>>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>>>>>> the
>>>>>>>> impact of punctuate is typically not observable by other processing
>>>>>>>> nodes
>>>>>>>> in the topology.
>>>>>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>>>>>> the
>>>>>>>> punctuate is all about being observable by downstream processing
>>>>>>>> nodes.
>>>>>>>>
>>>>>>>> A few releases back, we introduced record caches (DSL) and state
>>>>>>>> store
>>>>>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>>>>>> relating to
>>>>>>>> (2) where some users needed to control -- here: limit -- the
>>>>>>>> downstream
>>>>>>>> output rate of Kafka Streams because the downstream systems/apps
>>>>>>>> would not
>>>>>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>>>>>> their
>>>>>>>> scalability).  The argument for KIP-63, which notably did not
>>>>>>>> introduce a
>>>>>>>> "trigger" API, was that such an interaction with downstream systems
>>>>>>>> is an
>>>>>>>> operational concern;  it should not impact the processing *logic* of
>>>>>>>> your
>>>>>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>>>>>> API,
>>>>>>>> especially not the declarative DSL, with such operational concerns.
>>>>>>>>
>>>>>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>>>>>> sorry, I
>>>>>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>>>>>> observing that our conversation is moving more and more into the
>>>>>>>> direction
>>>>>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>>>>>> for
>>>>>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>>>>>> comments voiced here are about sth like "IF stream-time didn't
>>>>>>>> trigger
>>>>>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>>>>>> want
>>>>>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>>>>>> related
>>>>>>>> note, whatever we are discussing here will impact state store caches
>>>>>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>>>>>> should
>>>>>>>> clarify any such impact here.
>>>>>>>>
>>>>>>>> Switching topics slightly.
>>>>>>>>
>>>>>>>> Jay wrote:
>>>>>>>>
>>>>>>>> One thing I've always found super important for this kind of design
>>>>>>>> work
>>>>>>>> is to do a really good job of cataloging the landscape of use cases
>>>>>>>> and
>>>>>>>> how prevalent each one is.
>>>>>>>>
>>>>>>>> +1 to this, as others have already said.
>>>>>>>>
>>>>>>>> Here, let me highlight -- just in case -- that when we talked about
>>>>>>>> windowing use cases in the recent emails, the Processor API (where
>>>>>>>> `punctuate` resides) does not have any notion of windowing at
>>>>>>>> all.  If you
>>>>>>>> want to do windowing *in the Processor API*, you must do so manually
>>>>>>>> in
>>>>>>>> combination with window stores.  For this reason I'd suggest to
>>>>>>>> discuss use
>>>>>>>> cases not just in general, but also in view of how you'd do so in the
>>>>>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>>>>>> `punctuate`
>>>>>>>> does not impact the DSL at all, unless we add new functionality to
>>>>>>>> it.
>>>>>>>>
>>>>>>>> Jay wrote in his strawman example:
>>>>>>>>
>>>>>>>> You aggregate click and impression data for a reddit like site.
>>>>>>>> Every ten
>>>>>>>> minutes you want to output a ranked list of the top 10 articles
>>>>>>>> ranked by
>>>>>>>> clicks/impressions for each geographical area. I want to be able
>>>>>>>> run this
>>>>>>>> in steady state as well as rerun to regenerate results (or catch up
>>>>>>>> if it
>>>>>>>> crashes).
>>>>>>>>
>>>>>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>>>>>> we
>>>>>>>> argued that the reason for saying "every ten minutes" above is not
>>>>>>>> necessarily about because you want to output data *exactly* after ten
>>>>>>>> minutes, but that you want to perform an aggregation based on 10-
>>>>>>>> minute
>>>>>>>> windows of input data; i.e., the point is about specifying the input
>>>>>>>> for
>>>>>>>> your aggregation, not or less about when the results of the
>>>>>>>> aggregation
>>>>>>>> should be send downstream.  To take an extreme example, you could
>>>>>>>> disable
>>>>>>>> record caches and let your app output a downstream update for every
>>>>>>>> incoming input record.  If the last input record was from at minute 7
>>>>>>>> of 10
>>>>>>>> (for a 10-min window), then what your app would output at minute 10
>>>>>>>> would
>>>>>>>> be identical to what it had already emitted at minute 7 earlier
>>>>>>>> anyways.
>>>>>>>> This is particularly true when we take late-arriving data into
>>>>>>>> account:  if
>>>>>>>> a late record arrived at minute 13, your app would (by default) send
>>>>>>>> a new
>>>>>>>> update downstream, even though the "original" 10 minutes have already
>>>>>>>> passed.
>>>>>>>>
>>>>>>>> Jay wrote...:
>>>>>>>>
>>>>>>>> There are a couple of tricky things that seem to make this hard
>>>>>>>> with
>>>>>>>>
>>>>>>>> either
>>>>>>>>
>>>>>>>> of the options proposed:
>>>>>>>> 1. If I emit this data using event time I have the problem
>>>>>>>> described where
>>>>>>>> a geographical region with no new clicks or impressions will fail
>>>>>>>> to
>>>>>>>>
>>>>>>>> output
>>>>>>>>
>>>>>>>> results.
>>>>>>>>
>>>>>>>> ...and Arun Mathew wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> We window by the event time, but trigger punctuate in <punctuate
>>>>>>>> interval>
>>>>>>>> duration of system time, in the absence of an event crossing the
>>>>>>>> punctuate
>>>>>>>> event time.
>>>>>>>>
>>>>>>>> So, given what I wrote above about the status quo and what you can
>>>>>>>> already
>>>>>>>> do with it, is the concern that the state store cache doesn't give
>>>>>>>> you
>>>>>>>> *direct* control over "forcing an output after no later than X
>>>>>>>> seconds [of
>>>>>>>> processing-time]" but only indirect control through a cache
>>>>>>>> size?  (Note
>>>>>>>> that I am not dismissing the claims why this might be helpful.)
>>>>>>>>
>>>>>>>> Arun Mathew wrote:
>>>>>>>>
>>>>>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>>>>>> output the
>>>>>>>> event counts on each topic on each cluster aggregated over a 1
>>>>>>>> minute
>>>>>>>> window. We have to use event time to be able to cross check the
>>>>>>>> counts.
>>>>>>>>
>>>>>>>> But
>>>>>>>>
>>>>>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>>>>>> time in
>>>>>>>>
>>>>>>>> the
>>>>>>>>
>>>>>>>> absence of events. Otherwise the event counts for unexpired windows
>>>>>>>> would
>>>>>>>> be 0 which is bad.
>>>>>>>>
>>>>>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>>>>>> absence
>>>>>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>>>>>> Scala
>>>>>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>>>>>> root
>>>>>>>> cause that the downstream system interprets the absence of output in
>>>>>>>> a
>>>>>>>> particular way ("No output after 1 minute = I consider the output to
>>>>>>>> be
>>>>>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>>>>>> possible)
>>>>>>>> to correctly handle the difference between absence of output vs.
>>>>>>>> output of
>>>>>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>>>>>> but
>>>>>>>> want to understand the motivation better. :-)
>>>>>>>>
>>>>>>>> Also, to add some perspective, in some related discussions we talked
>>>>>>>> about
>>>>>>>> how a Kafka Streams application should not worry or not be coupled
>>>>>>>> unnecessarily with such interpretation specifics in a downstream
>>>>>>>> system's
>>>>>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>>>>>> more
>>>>>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>>>>>> than
>>>>>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>>>>>> and
>>>>>>>> downstream systems, including helping to reconcile the differences in
>>>>>>>> how
>>>>>>>> these systems interpret changes, updates, late-arriving data,
>>>>>>>> etc.  Kafka
>>>>>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>>>>>> processing from the specifics of downstream systems, thanks to
>>>>>>>> specific
>>>>>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>>>>>> this
>>>>>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>>>>>> but
>>>>>>>> it's currently awkward to use Connect for this", this might be a
>>>>>>>> problem we
>>>>>>>> can solve, too.)
>>>>>>>>
>>>>>>>> Switching topics slightly again.
>>>>>>>>
>>>>>>>> Thomas wrote:
>>>>>>>>
>>>>>>>> I'm not entirely convinced that a separate callback (option C)
>>>>>>>> is that messy (it could just be a default method with an empty
>>>>>>>> implementation), but if we wanted a single API to handle both
>>>>>>>> cases,
>>>>>>>> how about something like the following?
>>>>>>>>
>>>>>>>> enum Time {
>>>>>>>>     STREAM,
>>>>>>>>     CLOCK
>>>>>>>> }
>>>>>>>>
>>>>>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>>>>>> then
>>>>>>>> whatever the user is doing inside this method is a black box to Kafka
>>>>>>>> Streams (similar to how we have no idea what the user does inside a
>>>>>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>>>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>>>>>> won't
>>>>>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>>>>>> frequency than the processing-time action.  Or, we won't know whether
>>>>>>>> the
>>>>>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>>>>>> minute of
>>>>>>>> stream-time or 1-minute of processing-time, whichever comes
>>>>>>>> first").  That
>>>>>>>> said, I am not certain yet whether we would need such knowledge
>>>>>>>> because,
>>>>>>>> when using the Processor API, most of the work and decisions must be
>>>>>>>> done
>>>>>>>> by the user anyways.  It would matter though if the concept of
>>>>>>>> "triggers"
>>>>>>>> were to bubble up into the DSL because in the DSL the management of
>>>>>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>>>>>> Streams.
>>>>>>>>
>>>>>>>> [In any case, btw, we have the corner case where the user configured
>>>>>>>> the
>>>>>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>>>>>> extractor), at which point both punctuate variants are based on the
>>>>>>>> same
>>>>>>>> time semantics / timeline.]
>>>>>>>>
>>>>>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>>>>>> it
>>>>>>>> this far. :-)
>>>>>>>>
>>>>>>>> More than happy to hear your thoughts!
>>>>>>>> Michael
>>>>>>>>
>>>>>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>> Thanks Matthias.
>>>>>>>> Sure, will correct it right away.
>>>>>>>>
>>>>>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> Thanks for preparing this page!
>>>>>>>>
>>>>>>>> About terminology:
>>>>>>>>
>>>>>>>> You introduce the term "event time" -- but we should call this
>>>>>>>> "stream
>>>>>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>>>>>> this
>>>>>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>>>>>
>>>>>>>> Does this make sense to you?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>>>>
>>>>>>>> Thanks Ewen.
>>>>>>>>
>>>>>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>>>>>
>>>>>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>>>>
>>>>>>>>
>>>>>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>>>>>> public.
>>>>>>>>
>>>>>>>> --
>>>>>>>> Arun Mathew
>>>>>>>>
>>>>>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>      Arun,
>>>>>>>>
>>>>>>>>      I've given you permission to edit the wiki. Let me know if
>>>>>>>> you run
>>>>>>>>
>>>>>>>> into any
>>>>>>>>
>>>>>>>>      issues.
>>>>>>>>
>>>>>>>>      -Ewen
>>>>>>>>
>>>>>>>>      On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp> <am...@yahoo-corp.jp>
>>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>
>>>>>>>>      > Thanks Michal. I don\u2019t have the access yet [arunmathew88].
>>>>>>>> Should I
>>>>>>>>
>>>>>>>> be
>>>>>>>>
>>>>>>>>      > sending a separate mail for this?
>>>>>>>>      >
>>>>>>>>      > I thought one of the person following this thread would be
>>>>>>>> able to
>>>>>>>>
>>>>>>>> give me
>>>>>>>>
>>>>>>>>      > access.
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>>>>>>>      > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>>      > *Date: *Friday, April 7, 2017 at 17:16
>>>>>>>>      > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>>      > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>>>>> semantics
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > Hi Arun,
>>>>>>>>      >
>>>>>>>>      > I was thinking along the same lines as you, listing the use
>>>>>>>> cases
>>>>>>>>
>>>>>>>> on the
>>>>>>>>
>>>>>>>>      > wiki, but didn't find time to get around doing that yet.
>>>>>>>>      > Don't mind if you do it if you have access now.
>>>>>>>>      > I was thinking it would be nice if, once we have the use
>>>>>>>> cases
>>>>>>>>
>>>>>>>> listed,
>>>>>>>>
>>>>>>>>      > people could use likes to up-vote the use cases similar to
>>>>>>>> what
>>>>>>>>
>>>>>>>> they're
>>>>>>>>
>>>>>>>>      > working on.
>>>>>>>>      >
>>>>>>>>      > I should have a bit more time to action this in the next
>>>>>>>> few days,
>>>>>>>>
>>>>>>>> but
>>>>>>>>
>>>>>>>>      > happy for you to do it if you can beat me to it ;-)
>>>>>>>>      >
>>>>>>>>      > Cheers,
>>>>>>>>      > Michal
>>>>>>>>      >
>>>>>>>>      > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>>>>      >
>>>>>>>>      > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > Of course. I was thinking of a subpage where people can
>>>>>>>>
>>>>>>>> collaborate.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > Will do as per Michael\u2019s suggestion.
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > Regards,
>>>>>>>>      >
>>>>>>>>      > Arun Mathew
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>>> <
>>>>>>>>
>>>>>>>> matthias@confluent.io> wrote:
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >     Please share your Wiki-ID and a committer can give you
>>>>>>>> write
>>>>>>>>
>>>>>>>> access.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >     Btw: as you did not initiate the KIP, you should not
>>>>>>>> change the
>>>>>>>>
>>>>>>>> KIP
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     without the permission of the original author -- in
>>>>>>>> this case
>>>>>>>>
>>>>>>>> Michael.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >     So you might also just share your thought over the
>>>>>>>> mailing list
>>>>>>>>
>>>>>>>> and
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     Michael can update the KIP page. Or, as an alternative,
>>>>>>>> just
>>>>>>>>
>>>>>>>> create a
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     subpage for the KIP page.
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >     @Michael: WDYT?
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >     -Matthias
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>>>>      >
>>>>>>>>      >     > Hi Jay,
>>>>>>>>      >
>>>>>>>>      >     >           Thanks for the advise, I would like to list
>>>>>>>> down
>>>>>>>>
>>>>>>>> the use cases as
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > per your suggestion. But it seems I don't have write
>>>>>>>>
>>>>>>>> permission to the
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > Apache Kafka Confluent Space. Whom shall I request
>>>>>>>> for it?
>>>>>>>>      >
>>>>>>>>      >     >
>>>>>>>>      >
>>>>>>>>      >     > Regarding your last question. We are using a patch in
>>>>>>>> our
>>>>>>>>
>>>>>>>> production system
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > which does exactly this.
>>>>>>>>      >
>>>>>>>>      >     > We window by the event time, but trigger punctuate in
>>>>>>>>
>>>>>>>> <punctuate interval>
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > duration of system time, in the absence of an event
>>>>>>>> crossing
>>>>>>>>
>>>>>>>> the punctuate
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > event time.
>>>>>>>>      >
>>>>>>>>      >     >
>>>>>>>>      >
>>>>>>>>      >     > We are using Kafka Stream for our Audit Trail, where
>>>>>>>> we need
>>>>>>>>
>>>>>>>> to output the
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > event counts on each topic on each cluster aggregated
>>>>>>>> over a
>>>>>>>>
>>>>>>>> 1 minute
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > window. We have to use event time to be able to cross
>>>>>>>> check
>>>>>>>>
>>>>>>>> the counts. But
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > we need to trigger punctuate [aggregate event pushes]
>>>>>>>> by
>>>>>>>>
>>>>>>>> system time in the
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > absence of events. Otherwise the event counts for
>>>>>>>> unexpired
>>>>>>>>
>>>>>>>> windows would
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > be 0 which is bad.
>>>>>>>>      >
>>>>>>>>      >     >
>>>>>>>>      >
>>>>>>>>      >     > "Maybe a hybrid solution works: I window by event
>>>>>>>> time but
>>>>>>>>
>>>>>>>> trigger results
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > by system time for windows that have updated? Not
>>>>>>>> really sure
>>>>>>>>
>>>>>>>> the details
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > of making that work. Does that work? Are there
>>>>>>>> concrete
>>>>>>>>
>>>>>>>> examples where you
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > actually want the current behavior?"
>>>>>>>>      >
>>>>>>>>      >     >
>>>>>>>>      >
>>>>>>>>      >     > --
>>>>>>>>      >
>>>>>>>>      >     > With Regards,
>>>>>>>>      >
>>>>>>>>      >     >
>>>>>>>>      >
>>>>>>>>      >     > Arun Mathew
>>>>>>>>      >
>>>>>>>>      >     > Yahoo! JAPAN Corporation
>>>>>>>>      >
>>>>>>>>      >     >
>>>>>>>>      >
>>>>>>>>      >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>>>>>
>>>>>>>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >
>>>>>>>>      >
>>>>>>>>      >     >> Hi Jay,
>>>>>>>>      >
>>>>>>>>      >     >>
>>>>>>>>      >
>>>>>>>>      >     >> The hybrid solution is exactly what I expect and
>>>>>>>> need for
>>>>>>>>
>>>>>>>> our use cases
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> when dealing with telecom data.
>>>>>>>>      >
>>>>>>>>      >     >>
>>>>>>>>      >
>>>>>>>>      >     >> Thanks
>>>>>>>>      >
>>>>>>>>      >     >> Tianji
>>>>>>>>      >
>>>>>>>>      >     >>
>>>>>>>>      >
>>>>>>>>      >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>>>>>
>>>>>>>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>
>>>>>>>>      >
>>>>>>>>      >     >>> Hey guys,
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>> One thing I've always found super important for
>>>>>>>> this kind
>>>>>>>>
>>>>>>>> of design work
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> is
>>>>>>>>      >
>>>>>>>>      >     >>> to do a really good job of cataloging the landscape
>>>>>>>> of use
>>>>>>>>
>>>>>>>> cases and how
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> prevalent each one is. By that I mean not just
>>>>>>>> listing lots
>>>>>>>>
>>>>>>>> of uses, but
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> also grouping them into categories that
>>>>>>>> functionally need
>>>>>>>>
>>>>>>>> the same thing.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> In the absence of this it is very hard to reason
>>>>>>>> about
>>>>>>>>
>>>>>>>> design proposals.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> From the proposals so far I think we have a lot of
>>>>>>>>
>>>>>>>> discussion around
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> possible apis, but less around what the user needs
>>>>>>>> for
>>>>>>>>
>>>>>>>> different use
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> cases
>>>>>>>>      >
>>>>>>>>      >     >>> and how they would implement that using the api.
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>> Here is an example:
>>>>>>>>      >
>>>>>>>>      >     >>> You aggregate click and impression data for a
>>>>>>>> reddit like
>>>>>>>>
>>>>>>>> site. Every ten
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> minutes you want to output a ranked list of the top
>>>>>>>> 10
>>>>>>>>
>>>>>>>> articles ranked by
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> clicks/impressions for each geographical area. I
>>>>>>>> want to be
>>>>>>>>
>>>>>>>> able run this
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> in steady state as well as rerun to regenerate
>>>>>>>> results (or
>>>>>>>>
>>>>>>>> catch up if it
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> crashes).
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>> There are a couple of tricky things that seem to
>>>>>>>> make this
>>>>>>>>
>>>>>>>> hard with
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> either
>>>>>>>>      >
>>>>>>>>      >     >>> of the options proposed:
>>>>>>>>      >
>>>>>>>>      >     >>> 1. If I emit this data using event time I have the
>>>>>>>> problem
>>>>>>>>
>>>>>>>> described
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> where
>>>>>>>>      >
>>>>>>>>      >     >>> a geographical region with no new clicks or
>>>>>>>> impressions
>>>>>>>>
>>>>>>>> will fail to
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> output
>>>>>>>>      >
>>>>>>>>      >     >>> results.
>>>>>>>>      >
>>>>>>>>      >     >>> 2. If I emit this data using system time I have the
>>>>>>>> problem
>>>>>>>>
>>>>>>>> that when
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> reprocessing data my window may not be ten minutes
>>>>>>>> but 10
>>>>>>>>
>>>>>>>> hours if my
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> processing is very fast so it dramatically changes
>>>>>>>> the
>>>>>>>>
>>>>>>>> output.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>> Maybe a hybrid solution works: I window by event
>>>>>>>> time but
>>>>>>>>
>>>>>>>> trigger results
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> by system time for windows that have updated? Not
>>>>>>>> really
>>>>>>>>
>>>>>>>> sure the details
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> of making that work. Does that work? Are there
>>>>>>>> concrete
>>>>>>>>
>>>>>>>> examples where
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> you
>>>>>>>>      >
>>>>>>>>      >     >>> actually want the current behavior?
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>> -Jay
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>>>>>
>>>>>>>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> wrote:
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>>> Hi All,
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> Thanks for the KIP. We were also in need of a
>>>>>>>> mechanism to
>>>>>>>>
>>>>>>>> trigger
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> punctuate in the absence of events.
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> As I described in [
>>>>>>>>      >
>>>>>>>>      >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>>>>      >
>>>>>>>>      >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>>>>      >
>>>>>>>>      >     >>>> plugin.system.issuetabpanels:comment-
>>>>>>>> tabpanel#comment-
>>>>>>>>
>>>>>>>> 15926036
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> ],
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    - Our approached involved using the event time
>>>>>>>> by
>>>>>>>>
>>>>>>>> default.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    - The method to check if there is any punctuate
>>>>>>>> ready
>>>>>>>>
>>>>>>>> in the
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    PunctuationQueue is triggered via the any event
>>>>>>>>
>>>>>>>> received by the
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> stream
>>>>>>>>      >
>>>>>>>>      >     >>>>    tread, or at the polling intervals in the
>>>>>>>> absence of
>>>>>>>>
>>>>>>>> any events.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    - When we create Punctuate objects (which
>>>>>>>> contains the
>>>>>>>>
>>>>>>>> next event
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> time
>>>>>>>>      >
>>>>>>>>      >     >>>>    for punctuation and interval), we also record
>>>>>>>> the
>>>>>>>>
>>>>>>>> creation time
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> (system
>>>>>>>>      >
>>>>>>>>      >     >>>>    time).
>>>>>>>>      >
>>>>>>>>      >     >>>>    - While checking for maturity of Punctuate
>>>>>>>> Schedule by
>>>>>>>>      >
>>>>>>>>      >     >> mayBePunctuate
>>>>>>>>      >
>>>>>>>>      >     >>>>    method, we also check if the system clock has
>>>>>>>> elapsed
>>>>>>>>
>>>>>>>> the punctuate
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    interval since the schedule creation time.
>>>>>>>>      >
>>>>>>>>      >     >>>>    - In the absence of any event, or in the
>>>>>>>> absence of any
>>>>>>>>
>>>>>>>> event for
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> one
>>>>>>>>      >
>>>>>>>>      >     >>>>    topic in the partition group assigned to the
>>>>>>>> stream
>>>>>>>>
>>>>>>>> task, the system
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> time
>>>>>>>>      >
>>>>>>>>      >     >>>>    will elapse the interval and we trigger a
>>>>>>>> punctuate
>>>>>>>>
>>>>>>>> using the
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> expected
>>>>>>>>      >
>>>>>>>>      >     >>>>    punctuation event time.
>>>>>>>>      >
>>>>>>>>      >     >>>>    - we then create the next punctuation schedule
>>>>>>>> as
>>>>>>>>
>>>>>>>> punctuation event
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> time
>>>>>>>>      >
>>>>>>>>      >     >>>>    + punctuation interval, [again recording the
>>>>>>>> system
>>>>>>>>
>>>>>>>> time of creation
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> of
>>>>>>>>      >
>>>>>>>>      >     >>>> the
>>>>>>>>      >
>>>>>>>>      >     >>>>    schedule].
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>>>>> approach
>>>>>>>>
>>>>>>>> has pros and
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> cons.
>>>>>>>>      >
>>>>>>>>      >     >>>> Pros
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    - Punctuates will happen in <punctuate
>>>>>>>> interval> time
>>>>>>>>
>>>>>>>> duration at
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> max
>>>>>>>>      >
>>>>>>>>      >     >>> in
>>>>>>>>      >
>>>>>>>>      >     >>>>    terms of system time.
>>>>>>>>      >
>>>>>>>>      >     >>>>    - The semantics as a whole continues to revolve
>>>>>>>> around
>>>>>>>>
>>>>>>>> event time.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    - We can use the old data [old timestamps] to
>>>>>>>> rerun any
>>>>>>>>
>>>>>>>> experiments
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> or
>>>>>>>>      >
>>>>>>>>      >     >>>>    tests.
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> Cons
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    - In case the  <punctuate interval> is not a
>>>>>>>> time
>>>>>>>>
>>>>>>>> duration [say
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> logical
>>>>>>>>      >
>>>>>>>>      >     >>>>    time/event count], then the approach might not
>>>>>>>> be
>>>>>>>>
>>>>>>>> meaningful.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>    - In case there is a case where we have to wait
>>>>>>>> for an
>>>>>>>>
>>>>>>>> actual event
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> from
>>>>>>>>      >
>>>>>>>>      >     >>>>    a low event rate partition in the partition
>>>>>>>> group, this
>>>>>>>>
>>>>>>>> approach
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> will
>>>>>>>>      >
>>>>>>>>      >     >>>> jump
>>>>>>>>      >
>>>>>>>>      >     >>>>    the gun.
>>>>>>>>      >
>>>>>>>>      >     >>>>    - in case the event processing cannot catch up
>>>>>>>> with the
>>>>>>>>
>>>>>>>> event rate
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> and
>>>>>>>>      >
>>>>>>>>      >     >>>>    the expected timestamp events gets queued for
>>>>>>>> long
>>>>>>>>
>>>>>>>> time, this
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> approach
>>>>>>>>      >
>>>>>>>>      >     >>>>    might jump the gun.
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> I believe the above approach and discussion goes
>>>>>>>> close to
>>>>>>>>
>>>>>>>> the approach
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> A.
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> -----------
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> I like the idea of having an even count based
>>>>>>>> punctuate.
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> -----------
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> I agree with the discussion around approach C,
>>>>>>>> that we
>>>>>>>>
>>>>>>>> should provide
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> the
>>>>>>>>      >
>>>>>>>>      >     >>>> user with the option to choose system time or
>>>>>>>> event time
>>>>>>>>
>>>>>>>> based
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> punctuates.
>>>>>>>>      >
>>>>>>>>      >     >>>> But I believe that the user predominantly wants to
>>>>>>>> use
>>>>>>>>
>>>>>>>> event time while
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> not
>>>>>>>>      >
>>>>>>>>      >     >>>> missing out on regular punctuates due to event
>>>>>>>> delays or
>>>>>>>>
>>>>>>>> event
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> absences.
>>>>>>>>      >
>>>>>>>>      >     >>>> Hence a complex punctuate option as Matthias
>>>>>>>> mentioned
>>>>>>>>
>>>>>>>> (quoted below)
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> would
>>>>>>>>      >
>>>>>>>>      >     >>>> be most apt.
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> "- We might want to add "complex" schedules later
>>>>>>>> on
>>>>>>>>
>>>>>>>> (like, punctuate
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> on
>>>>>>>>      >
>>>>>>>>      >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>>>>> time
>>>>>>>>
>>>>>>>> whatever comes
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> first)."
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> -----------
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> I think I read somewhere that Kafka Streams
>>>>>>>> started with
>>>>>>>>
>>>>>>>> System Time as
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> the
>>>>>>>>      >
>>>>>>>>      >     >>>> punctuation standard, but was later changed to
>>>>>>>> Event Time.
>>>>>>>>
>>>>>>>> I guess
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> there
>>>>>>>>      >
>>>>>>>>      >     >>>> would be some good reason behind it. As Kafka
>>>>>>>> Streams want
>>>>>>>>
>>>>>>>> to evolve
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> more
>>>>>>>>      >
>>>>>>>>      >     >>>> on the Stream Processing front, I believe the
>>>>>>>> emphasis on
>>>>>>>>
>>>>>>>> event time
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> would
>>>>>>>>      >
>>>>>>>>      >     >>>> remain quite strong.
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> With Regards,
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> Arun Mathew
>>>>>>>>      >
>>>>>>>>      >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>>>>>
>>>>>>>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> wrote:
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> Yeah I like PuncutationType much better; I just
>>>>>>>> threw
>>>>>>>>
>>>>>>>> Time out there
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>>>>> still
>>>>>>>>
>>>>>>>> think it's
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> worth considering what this buys us over an
>>>>>>>> additional
>>>>>>>>
>>>>>>>> callback. I
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> foresee a number of punctuate implementations
>>>>>>>> following
>>>>>>>>
>>>>>>>> this pattern:
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> public void punctuate(PunctuationType type) {
>>>>>>>>      >
>>>>>>>>      >     >>>>>     switch (type) {
>>>>>>>>      >
>>>>>>>>      >     >>>>>         case EVENT_TIME:
>>>>>>>>      >
>>>>>>>>      >     >>>>>             methodA();
>>>>>>>>      >
>>>>>>>>      >     >>>>>             break;
>>>>>>>>      >
>>>>>>>>      >     >>>>>         case SYSTEM_TIME:
>>>>>>>>      >
>>>>>>>>      >     >>>>>             methodB();
>>>>>>>>      >
>>>>>>>>      >     >>>>>             break;
>>>>>>>>      >
>>>>>>>>      >     >>>>>     }
>>>>>>>>      >
>>>>>>>>      >     >>>>> }
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> I guess one advantage of this approach is we
>>>>>>>> could add
>>>>>>>>
>>>>>>>> additional
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> punctuation types later in a backwards compatible
>>>>>>>> way
>>>>>>>>
>>>>>>>> (like event
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> count
>>>>>>>>      >
>>>>>>>>      >     >>>>> as you mentioned).
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> -Tommy
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>>>>> Sax wrote:
>>>>>>>>      >
>>>>>>>>      >     >>>>>> That sounds promising.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>> I am just wondering if `Time` is the best name.
>>>>>>>> Maybe we
>>>>>>>>
>>>>>>>> want to
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> add
>>>>>>>>      >
>>>>>>>>      >     >>>>>> other non-time based punctuations at some point
>>>>>>>> later. I
>>>>>>>>
>>>>>>>> would
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>> suggest
>>>>>>>>      >
>>>>>>>>      >     >>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>> enum PunctuationType {
>>>>>>>>      >
>>>>>>>>      >     >>>>>>   EVENT_TIME,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>   SYSTEM_TIME,
>>>>>>>>      >
>>>>>>>>      >     >>>>>> }
>>>>>>>>      >
>>>>>>>>      >     >>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>> or similar. Just to keep the door open -- it's
>>>>>>>> easier to
>>>>>>>>
>>>>>>>> add new
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>> stuff
>>>>>>>>      >
>>>>>>>>      >     >>>>>> if the name is more generic.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>> -Matthias
>>>>>>>>      >
>>>>>>>>      >     >>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> I agree that the framework providing and
>>>>>>>> managing the
>>>>>>>>
>>>>>>>> notion of
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> stream
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> time is valuable and not something we would
>>>>>>>> want to
>>>>>>>>
>>>>>>>> delegate to
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> the
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>>>>> separate
>>>>>>>>
>>>>>>>> callback
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> (option
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> C)
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> is that messy (it could just be a default
>>>>>>>> method with
>>>>>>>>
>>>>>>>> an empty
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> implementation), but if we wanted a single API
>>>>>>>> to
>>>>>>>>
>>>>>>>> handle both
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> cases,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> how about something like the following?
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> enum Time {
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>    STREAM,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>    CLOCK
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> }
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> Then on ProcessorContext:
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>>>>> We could
>>>>>>>>
>>>>>>>> allow
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> this
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> to
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> be called once for each value of time to mix
>>>>>>>>
>>>>>>>> approaches.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> Then the Processor API becomes:
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>>>>>
>>>>>>>> schedule resulted
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> in
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> this call.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> Thoughts?
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>>>>> Sax
>>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> I was thinking about the four options you
>>>>>>>> proposed in
>>>>>>>>
>>>>>>>> more
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> details
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> and
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> this are my thoughts:
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> (A) You argue, that users can still
>>>>>>>> "punctuate" on
>>>>>>>>
>>>>>>>> event-time
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> via
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> process(), but I am not sure if this is
>>>>>>>> possible.
>>>>>>>>
>>>>>>>> Note, that
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> users
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> only
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> get record timestamps via context.timestamp().
>>>>>>>> Thus,
>>>>>>>>
>>>>>>>> users
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> would
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> need
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> to
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> track the time progress per partition (based
>>>>>>>> on the
>>>>>>>>
>>>>>>>> partitions
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> they
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> obverse via context.partition(). (This alone
>>>>>>>> puts a
>>>>>>>>
>>>>>>>> huge burden
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> on
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> the
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> user by itself.) However, users are not
>>>>>>>> notified at
>>>>>>>>
>>>>>>>> startup
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> what
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> partitions are assigned, and user are not
>>>>>>>> notified
>>>>>>>>
>>>>>>>> when
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> partitions
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> get
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> revoked. Because this information is not
>>>>>>>> available,
>>>>>>>>
>>>>>>>> it's not
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> possible
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> to
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> "manually advance" stream-time, and thus
>>>>>>>> event-time
>>>>>>>>
>>>>>>>> punctuation
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> within
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> process() seems not to be possible -- or do
>>>>>>>> you see a
>>>>>>>>
>>>>>>>> way to
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> get
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> it
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> done? And even if, it might still be too
>>>>>>>> clumsy to
>>>>>>>>
>>>>>>>> use.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> (B) This does not allow to mix both
>>>>>>>> approaches, thus
>>>>>>>>
>>>>>>>> limiting
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> what
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> users
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> can do.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> (C) This should give all flexibility we need.
>>>>>>>> However,
>>>>>>>>
>>>>>>>> just
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> adding
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> one
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> more method seems to be a solution that is too
>>>>>>>> simple
>>>>>>>>
>>>>>>>> (cf my
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> comments
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> below).
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>>>>> sure how
>>>>>>>>
>>>>>>>> a user
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> could
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> enable system-time and event-time punctuation
>>>>>>>> in
>>>>>>>>
>>>>>>>> parallel.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> Overall options (C) seems to be the most
>>>>>>>> promising
>>>>>>>>
>>>>>>>> approach to
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> me.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> Because I also favor a clean API, we might
>>>>>>>> keep
>>>>>>>>
>>>>>>>> current
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> punctuate()
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>>>>> at some
>>>>>>>>
>>>>>>>> later
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> point
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> when
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> people use the "new punctuate API".
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> Couple of follow up questions:
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> - I am wondering, if we should have two
>>>>>>>> callback
>>>>>>>>
>>>>>>>> methods or
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> just
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> one
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> (ie, a unified for system and event time
>>>>>>>> punctuation
>>>>>>>>
>>>>>>>> or one for
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> each?).
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> - If we have one, how can the user figure out,
>>>>>>>> which
>>>>>>>>
>>>>>>>> condition
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> did
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> trigger?
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> - How would the API look like, for registering
>>>>>>>>
>>>>>>>> different
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> punctuate
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> - We might want to add "complex" schedules
>>>>>>>> later on
>>>>>>>>
>>>>>>>> (like,
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> punctuate
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> on
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>>>>> system-time
>>>>>>>>
>>>>>>>> whatever
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> comes
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> first). I don't say we should add this right
>>>>>>>> away, but
>>>>>>>>
>>>>>>>> we might
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> want
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> to
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> define the API in a way, that it allows
>>>>>>>> extensions
>>>>>>>>
>>>>>>>> like this
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> later
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> on,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> without redesigning the API (ie, the API
>>>>>>>> should be
>>>>>>>>
>>>>>>>> designed
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> extensible)
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> - Did you ever consider count-based
>>>>>>>> punctuation?
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> I understand, that you would like to solve a
>>>>>>>> simple
>>>>>>>>
>>>>>>>> problem,
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> but
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> we
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> learned from the past, that just "adding some
>>>>>>>> API"
>>>>>>>>
>>>>>>>> quickly
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> leads
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> to a
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> not very well defined API that needs time
>>>>>>>> consuming
>>>>>>>>
>>>>>>>> clean up
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> later on
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>>>>> holistic
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> punctuation
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> KIP
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> with this from the beginning on to avoid later
>>>>>>>> painful
>>>>>>>>      >
>>>>>>>>      >     >> redesign.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> -Matthias
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> Thanks Thomas,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> I'm also wary of changing the existing
>>>>>>>> semantics of
>>>>>>>>      >
>>>>>>>>      >     >> punctuate,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> for
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> backward compatibility reasons, although I
>>>>>>>> like the
>>>>>>>>      >
>>>>>>>>      >     >> conceptual
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> simplicity of that option.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>>>>> a way,
>>>>>>>>
>>>>>>>> uglier.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> I
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> added
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> this to the KIP now as option (C).
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>>>>> more
>>>>>>>>
>>>>>>>> flexible,
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> as
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> it
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> allows
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> you to return any value, you're not limited
>>>>>>>> to event
>>>>>>>>
>>>>>>>> time or
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> system
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> time
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> (although I don't see an actual use case
>>>>>>>> where you
>>>>>>>>
>>>>>>>> might need
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> anything
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> else then those two). Hence I also proposed
>>>>>>>> the
>>>>>>>>
>>>>>>>> option to
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> allow
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> users
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>>>>> for
>>>>>>>>
>>>>>>>> them given
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> the
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> presence or absence of messages, much like
>>>>>>>> they can
>>>>>>>>
>>>>>>>> decide
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> what
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> msg
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> time
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>>>>> What do
>>>>>>>>
>>>>>>>> you
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> think
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> about
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> that? This is probably most flexible but also
>>>>>>>> most
>>>>>>>>      >
>>>>>>>>      >     >> complicated.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> All comments appreciated.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> Cheers,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> Michal
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> Although I fully agree we need a way to
>>>>>>>> trigger
>>>>>>>>
>>>>>>>> periodic
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> processing
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> that is independent from whether and when
>>>>>>>> messages
>>>>>>>>
>>>>>>>> arrive,
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> I'm
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> not sure
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> I like the idea of changing the existing
>>>>>>>> semantics
>>>>>>>>
>>>>>>>> across
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> the
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> board.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> What if we added an additional callback to
>>>>>>>> Processor
>>>>>>>>
>>>>>>>> that
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> can
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> be
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>>>>> always
>>>>>>>>
>>>>>>>> called at
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> fixed, wall
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>>>>> have to
>>>>>>>>
>>>>>>>> give
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> up
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> the
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> notion
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> of stream time to be able to do periodic
>>>>>>>> processing.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>>>>> Borowiecki
>>>>>>>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> Hi all,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>>>>>
>>>>>>>> punctuate
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> semantics
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>>>>>
>>>>>>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/ confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>>>>>
>>>>>>>> 138%
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> 3A+C
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> hange+
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> punctuate+semantics>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> .
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> Appreciating there can be different views
>>>>>>>> on
>>>>>>>>
>>>>>>>> system-time
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> vs
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> event-
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> time
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>>>>> case and
>>>>>>>>
>>>>>>>> the
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> importance of
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>>>>> I've
>>>>>>>>
>>>>>>>> left it
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> quite
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> open
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> and
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>>>>>
>>>>>>>> progresses.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> Thanks,
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>>>>> Michal
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> --
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>     Tommy Becker
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>     Senior Software Engineer
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>     tivo.com
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> ________________________________
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> This email and any attachments may contain
>>>>>>>> confidential
>>>>>>>>
>>>>>>>> and
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> privileged material for the sole use of the
>>>>>>>> intended
>>>>>>>>
>>>>>>>> recipient.
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> Any
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> review, copying, or distribution of this email
>>>>>>>> (or any
>>>>>>>>      >
>>>>>>>>      >     >> attachments)
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> by others is prohibited. If you are not the
>>>>>>>> intended
>>>>>>>>
>>>>>>>> recipient,
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> please contact the sender immediately and
>>>>>>>> permanently
>>>>>>>>
>>>>>>>> delete this
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> email and any attachments. No employee or agent
>>>>>>>> of TiVo
>>>>>>>>
>>>>>>>> Inc. is
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> authorized to conclude any binding agreement on
>>>>>>>> behalf
>>>>>>>>
>>>>>>>> of TiVo
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> Inc.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>>>>> only be
>>>>>>>>
>>>>>>>> made by a
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>>> signed written agreement.
>>>>>>>>      >
>>>>>>>>      >     >>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> --
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>     Tommy Becker
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>     Senior Software Engineer
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>     tivo.com
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> ________________________________
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> This email and any attachments may contain
>>>>>>>> confidential
>>>>>>>>
>>>>>>>> and
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> privileged
>>>>>>>>      >
>>>>>>>>      >     >>>>> material for the sole use of the intended
>>>>>>>> recipient. Any
>>>>>>>>
>>>>>>>> review,
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>> copying,
>>>>>>>>      >
>>>>>>>>      >     >>>>> or distribution of this email (or any
>>>>>>>> attachments) by
>>>>>>>>
>>>>>>>> others is
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>> prohibited.
>>>>>>>>      >
>>>>>>>>      >     >>>>> If you are not the intended recipient, please
>>>>>>>> contact the
>>>>>>>>
>>>>>>>> sender
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> immediately and permanently delete this email and
>>>>>>>> any
>>>>>>>>
>>>>>>>> attachments. No
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>>>>> conclude
>>>>>>>>
>>>>>>>> any binding
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>>>>> Binding
>>>>>>>>
>>>>>>>> agreements with
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      >     >> TiVo
>>>>>>>>      >
>>>>>>>>      >     >>>>> Inc. may only be made by a signed written
>>>>>>>> agreement.
>>>>>>>>      >
>>>>>>>>      >     >>>>>
>>>>>>>>      >
>>>>>>>>      >     >>>>
>>>>>>>>      >
>>>>>>>>      >     >>>
>>>>>>>>      >
>>>>>>>>      >     >>
>>>>>>>>      >
>>>>>>>>      >     >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > --
>>>>>>>>      >
>>>>>>>>      > <http://www.openbet.com/> <http://www.openbet.com/>
>>>>>>>>
>>>>>>>>      >
>>>>>>>>      > *Michal Borowiecki*
>>>>>>>>      >
>>>>>>>>      > *Senior Software Engineer L4*
>>>>>>>>      >
>>>>>>>>      > *T: *
>>>>>>>>      >
>>>>>>>>      > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>>>>>      >
>>>>>>>>      > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > *E: *
>>>>>>>>      >
>>>>>>>>      > michal.borowiecki@openbet.com
>>>>>>>>      >
>>>>>>>>      > *W: *
>>>>>>>>      >
>>>>>>>>      > www.openbet.com
>>>>>>>>      >
>>>>>>>>      > *OpenBet Ltd*
>>>>>>>>      >
>>>>>>>>      > Chiswick Park Building 9
>>>>>>>>      >
>>>>>>>>      > 566 Chiswick High Rd
>>>>>>>>      >
>>>>>>>>      > London
>>>>>>>>      >
>>>>>>>>      > W4 5XT
>>>>>>>>      >
>>>>>>>>      > UK
>>>>>>>>      >
>>>>>>>>      > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      > This message is confidential and intended only for the
>>>>>>>> addressee.
>>>>>>>>
>>>>>>>> If you
>>>>>>>>
>>>>>>>>      > have received this message in error, please immediately
>>>>>>>> notify the
>>>>>>>>      > postmaster@openbet.com and delete it from your system as
>>>>>>>> well as
>>>>>>>>
>>>>>>>> any
>>>>>>>>
>>>>>>>>      > copies. The content of e-mails as well as traffic data may
>>>>>>>> be
>>>>>>>>
>>>>>>>> monitored by
>>>>>>>>
>>>>>>>>      > OpenBet for employment and security purposes. To protect
>>>>>>>> the
>>>>>>>>
>>>>>>>> environment
>>>>>>>>
>>>>>>>>      > please do not print this e-mail unless necessary. OpenBet
>>>>>>>> Ltd.
>>>>>>>>
>>>>>>>> Registered
>>>>>>>>
>>>>>>>>      > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>>>>> London,
>>>>>>>>
>>>>>>>> W4 5XT,
>>>>>>>>
>>>>>>>>      > United Kingdom. A company registered in England and Wales.
>>>>>>>>
>>>>>>>> Registered no.
>>>>>>>>
>>>>>>>>      > 3134634. VAT no. GB927523612
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>      >
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>>
>>>>>>>>
>>>>>>>>      Tommy Becker
>>>>>>>>
>>>>>>>>      Senior Software Engineer
>>>>>>>>
>>>>>>>>      O +1 919.460.4747 <%28919%29%20460-4747>
>>>>>>>>
>>>>>>>>
>>>>>>>>      tivo.com
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> ________________________________
>>>>>>>>
>>>>>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>> <http://www.openbet.com/> Michal Borowiecki
>>>>>>>> Senior Software Engineer L4
>>>>>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>> -- 
>>>>> Signature
>>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>>> Senior Software Engineer L4
>>>>> 	T: 	+44 208 742 1600
>>>>>
>>>>> 	
>>>>> 	+44 203 249 8448
>>>>>
>>>>> 	
>>>>> 	
>>>>> 	E: 	michal.borowiecki@openbet.com
>>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>>
>>>>> 	
>>>>> 	OpenBet Ltd
>>>>>
>>>>> 	Chiswick Park Building 9
>>>>>
>>>>> 	566 Chiswick High Rd
>>>>>
>>>>> 	London
>>>>>
>>>>> 	W4 5XT
>>>>>
>>>>> 	UK
>>>>>
>>>>> 	
>>>>> <https://www.openbet.com/email_promo>
>>>>>
>>>>> This message is confidential and intended only for the addressee. If
>>>>> you have received this message in error, please immediately notify the
>>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>>> from your system as well as any copies. The content of e-mails as well
>>>>> as traffic data may be monitored by OpenBet for employment and
>>>>> security purposes. To protect the environment please do not print this
>>>>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>>>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>>>> company registered in England and Wales. Registered no. 3134634. VAT
>>>>> no. GB927523612
>>>>>
>>>> -- 
>>>> Signature
>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>> Senior Software Engineer L4
>>>> 	T: 	+44 208 742 1600
>>>>
>>>> 	
>>>> 	+44 203 249 8448
>>>>
>>>> 	
>>>> 	
>>>> 	E: 	michal.borowiecki@openbet.com
>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>
>>>> 	
>>>> 	OpenBet Ltd
>>>>
>>>> 	Chiswick Park Building 9
>>>>
>>>> 	566 Chiswick High Rd
>>>>
>>>> 	London
>>>>
>>>> 	W4 5XT
>>>>
>>>> 	UK
>>>>
>>>> 	
>>>> <https://www.openbet.com/email_promo>
>>>>
>>>> This message is confidential and intended only for the addressee. If you
>>>> have received this message in error, please immediately notify the
>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>> from your system as well as any copies. The content of e-mails as well
>>>> as traffic data may be monitored by OpenBet for employment and security
>>>> purposes. To protect the environment please do not print this e-mail
>>>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>>> registered in England and Wales. Registered no. 3134634. VAT no.
>>>> GB927523612
>>>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If you
>> have received this message in error, please immediately notify the
>> postmaster@openbet.com <ma...@openbet.com> and delete it
>> from your system as well as any copies. The content of e-mails as well
>> as traffic data may be monitored by OpenBet for employment and security
>> purposes. To protect the environment please do not print this e-mail
>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>> registered in England and Wales. Registered no. 3134634. VAT no.
>> GB927523612
>>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
A simple example would be some dashboard app, that needs to get
"current" status in regular time intervals (ie, and real-time app).

Or something like a "scheduler" -- think "cron job" application.


-Matthias

On 4/24/17 2:23 AM, Michal Borowiecki wrote:
> Hi Matthias,
> 
> I agree it's difficult to reason about the hybrid approach, I certainly
> found it hard and I'm totally on board with the mantra.
> 
> I'd be happy to limit the scope of this KIP to add system-time
> punctuation semantics (in addition to existing stream-time semantics)
> and leave more complex schemes for users to implement on top of that.
> 
> Further additional PunctuationTypes, could then be added by future KIPs,
> possibly including the hybrid approach once it has been given more thought.
> 
>> There are real-time applications, that want to get
>> callbacks in regular system-time intervals (completely independent from
>> stream-time).
> Can you please describe what they are, so that I can put them on the
> wiki for later reference?
> 
> Thanks,
> 
> Michal
> 
> 
> On 23/04/17 21:27, Matthias J. Sax wrote:
>> Hi,
>>
>> I do like Damian's API proposal about the punctuation callback function.
>>
>> I also did reread the KIP and thought about the semantics we want to
>> provide.
>>
>>> Given the above, I don't see a reason any more for a separate system-time based punctuation.
>> I disagree here. There are real-time applications, that want to get
>> callbacks in regular system-time intervals (completely independent from
>> stream-time). Thus we should allow this -- if we really follow the
>> "hybrid" approach, this could be configured with stream-time interval
>> infinite and delay whatever system-time punctuation interval you want to
>> have. However, I would like to add a proper API for this and do this
>> configuration under the hood (that would allow one implementation within
>> all kind of branching for different cases).
>>
>> Thus, we definitely should have PunctutionType#StreamTime and
>> #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
>> a fan of your latest API proposal.
>>
>>
>> About the hybrid approach in general. On the one hand I like it, on the
>> other hand, it seems to be rather (1) complicated (not necessarily from
>> an implementation point of view, but for people to understand it) and
>> (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
>>
>>> It may appear complicated at first but I do think these semantics will
>>> still be more understandable to users than having 2 separate punctuation
>>> schedules/callbacks with different PunctuationTypes.
>> This statement only holds if you apply strong assumptions that I don't
>> believe hold in general -- see (2) for details -- and I think it is
>> harder than you assume to reason about the hybrid approach in general.
>> IMHO, the hybrid approach is a "false friend" that seems to be easy to
>> reason about...
>>
>>
>> (1) Streams always embraced "easy to use" and we should really be
>> careful to keep it this way. On the other hand, as we are talking about
>> changes to PAPI, it won't affect DSL users (DSL does not use punctuation
>> at all at the moment), and thus, the "easy to use" mantra might not be
>> affected, while it will allow advanced users to express more complex stuff.
>>
>> I like the mantra: "make simple thing easy and complex things possible".
>>
>> (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
>> implicit assumption that even-time progresses at the same "speed" as
>> system-time during regular processing. This implies the assumption that
>> a slower progress in stream-time indicates the absence of input events
>> (and that later arriving input events will have a larger event-time with
>> high probability). Even if this might be true for some use cases, I
>> doubt it holds in general. Assume that you get a spike in traffic and
>> for some reason stream-time does advance slowly because you have more
>> records to process. This might trigger a system-time based punctuation
>> call even if this seems not to be intended. I strongly believe that it
>> is not easy to reason about the semantics of the hybrid approach (even
>> if the intentional semantics would be super useful -- but I doubt that
>> we get want we ask for).
>>
>> Thus, I also believe that one might need different "configuration"
>> values for the hybrid approach if you run the same code for different
>> scenarios: regular processing, re-processing, catching up scenario. And
>> as the term "configuration" implies, we might be better off to not mix
>> configuration with business logic that is expressed via code.
>>
>>
>> One more comment: I also don't think that the hybrid approach is
>> deterministic as claimed in the use-case subpage. I understand the
>> reasoning and agree, that it is deterministic if certain assumptions
>> hold -- compare above -- and if configured correctly. But strictly
>> speaking it's not because there is a dependency on system-time (and
>> IMHO, if system-time is involved it cannot be deterministic by definition).
>>
>>
>>> I see how in theory this could be implemented on top of the 2 punctuate
>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>> the other system-time based) but it would be a much more complicated
>>> scheme and I don't want to suggest that.
>> I agree that expressing the intended hybrid semantics is harder if we
>> offer only #StreamTime and #SystemTime punctuation. However, I also
>> believe that the hybrid approach is a "false friend" with regard to
>> reasoning about the semantics (it indicates that it more easy as it is
>> in reality). Therefore, we might be better off to not offer the hybrid
>> approach and make it clear to a developed, that it is hard to mix
>> #StreamTime and #SystemTime in a semantically sound way.
>>
>>
>> Looking forward to your feedback. :)
>>
>> -Matthias
>>
>>
>>
>>
>> On 4/22/17 11:43 AM, Michal Borowiecki wrote:
>>> Hi all,
>>>
>>> Looking for feedback on the functional interface approach Damian
>>> proposed. What do people think?
>>>
>>> Further on the semantics of triggering punctuate though:
>>>
>>> I ran through the 2 use cases that Arun had kindly put on the wiki
>>> (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
>>> in my head and on a whiteboard and I can't find a better solution than
>>> the "hybrid" approach he had proposed.
>>>
>>> I see how in theory this could be implemented on top of the 2 punctuate
>>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>>> the other system-time based) but it would be a much more complicated
>>> scheme and I don't want to suggest that.
>>>
>>> However, to add to the hybrid algorithm proposed, I suggest one
>>> parameter to that: a tolerance period, expressed in milliseconds
>>> system-time, after which the punctuation will be invoked in case the
>>> stream-time advance hasn't triggered it within the requested interval
>>> since the last invocation of punctuate (as recorded in system-time)
>>>
>>> This would allow a user-defined tolerance for late arriving events. The
>>> trade off would be left for the user to decide: regular punctuation in
>>> the case of absence of events vs allowing for records arriving late or
>>> some build-up due to processing not catching up with the event rate.
>>> In the one extreme, this tolerance could be set to infinity, turning
>>> hybrid into simply stream-time based punctuate, like we have now. In the
>>> other extreme, the tolerance could be set to 0, resulting in a
>>> system-time upper bound on the effective punctuation interval.
>>>
>>> Given the above, I don't see a reason any more for a separate
>>> system-time based punctuation. The "hybrid" approach with 0ms tolerance
>>> would under normal operation trigger at regular intervals wrt the
>>> system-time, except in cases of re-play/catch-up, where the stream time
>>> advances faster than system time. In these cases punctuate would happen
>>> more often than the specified interval wrt system time. However, the
>>> use-cases that need system-time punctuations (that I've seen at least)
>>> really only have a need for an upper bound on punctuation delay but
>>> don't need a lower bound.
>>>
>>> To that effect I'd propose the api to be as follows, on ProcessorContext:
>>>
>>> schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
>>>
>>> schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
>>>
>>> Punctuation is triggered when either:
>>> - the stream time advances past the (stream time of the previous
>>> punctuation) + interval;
>>> - or (iff the toleranceInterval is set) when the system time advances
>>> past the (system time of the previous punctuation) + interval +
>>> toleranceInterval
>>>
>>> In either case:
>>> - we trigger punctuate passing as the argument the stream time at which
>>> the current punctuation was meant to happen
>>> - next punctuate is scheduled at (stream time at which the current
>>> punctuation was meant to happen) + interval
>>>
>>> It may appear complicated at first but I do think these semantics will
>>> still be more understandable to users than having 2 separate punctuation
>>> schedules/callbacks with different PunctuationTypes.
>>>
>>>
>>>
>>> PS. Having re-read this, maybe the following alternative would be easier
>>> to understand (WDYT?):
>>>
>>> schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
>>>
>>> schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
>>>
>>> Punctuation is triggered when either:
>>> - the stream time advances past the (stream time of the previous
>>> punctuation) + streamTimeInterval;
>>> - or (iff systemTimeUpperBound is set) when the system time advances
>>> past the (system time of the previous punctuation) + systemTimeUpperBound
>>>
>>> Awaiting comments.
>>>
>>> Thanks,
>>> Michal
>>>
>>> On 21/04/17 16:56, Michal Borowiecki wrote:
>>>> Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>>>> in favour of something that doesn't return a record. Not a problem though.
>>>>
>>>>
>>>> On 21/04/17 16:32, Damian Guy wrote:
>>>>> Thanks Michal,
>>>>> I agree Transformer.punctuate should also be void, but we can deprecate
>>>>> that too in favor of the new interface.
>>>>>
>>>>> Thanks for the javadoc PR!
>>>>>
>>>>> Cheers,
>>>>> Damian
>>>>>
>>>>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>
>>>>>> Yes, that looks better to me.
>>>>>>
>>>>>> Note that punctuate on Transformer is currently returning a record, but I
>>>>>> think it's ok to have all output records be sent via
>>>>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>>>>> multiple records from one invocation of punctuate.
>>>>>>
>>>>>> This way it's consistent between Processor and Transformer.
>>>>>>
>>>>>>
>>>>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>>>>> there:
>>>>>>
>>>>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>>>>
>>>>>> and PR: https://github.com/apache/kafka/pull/2884
>>>>>>
>>>>>> Cheers,
>>>>>>
>>>>>> Michal
>>>>>> On 20/04/17 18:55, Damian Guy wrote:
>>>>>>
>>>>>> Hi Michal,
>>>>>>
>>>>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>>>>> the API.
>>>>>> 1. deprecate the punctuate method on Processor
>>>>>> 2. create a new Functional Interface just for Punctuation, something like:
>>>>>> interface Punctuator {
>>>>>>     void punctuate(long timestamp)
>>>>>> }
>>>>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>>>>> interval, PunctuationType type, Punctuator callback)
>>>>>> 4. deprecate the existing schedule function
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>> Thanks,
>>>>>> Damian
>>>>>>
>>>>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>>
>>>>>>> Hi Thomas,
>>>>>>>
>>>>>>> I would say our use cases fall in the same category as yours.
>>>>>>>
>>>>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>>>>
>>>>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>>>>> of design:
>>>>>>>
>>>>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>>>>
>>>>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>>>>
>>>>>>> Since the output field X is derived in some non-trivial way, we don't
>>>>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>>>>> another input field Z).
>>>>>>>
>>>>>>> So we have kv stores with the records and an additional kv store with
>>>>>>> timestamp->id mapping which act like an index where we periodically do a
>>>>>>> ranged query.
>>>>>>>
>>>>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>>>>> work when there were no regular msgs on the input topic.
>>>>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>>>>> outside we created a "ticker" that produced msgs once per second onto
>>>>>>> another topic and fed it into the same topology to trigger punctuate.
>>>>>>> This didn't work either, which was much more surprising to us at the
>>>>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>>>>> *all* input partitions receive messages regularly.
>>>>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>>>>> is consumed by the second topology and is its only input topic. There's a
>>>>>>> transformer on that topic which populates and updates the time-based
>>>>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>>>>> elapsed, the record id is sent to the main transformer, which
>>>>>>> updates/deletes the record from the main kv store and forwards the
>>>>>>> transformed record to the output topic.
>>>>>>>
>>>>>>> To me this setup feels horrendously complicated for what it does.
>>>>>>>
>>>>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>>>>> However, the ticks don't feel so hacky when you realise they give you
>>>>>>> some hypothetical benefits in predictability. You can reprocess the
>>>>>>> messages in a reproducible manner, since the topologies use event-time,
>>>>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>>>>> reprocessing).
>>>>>>> To make that work though, we would have to have the stream time advance
>>>>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>>>>> presence of messages on the other input topic.
>>>>>>>
>>>>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>>>>> and the hybrid would work to simplify this a lot.
>>>>>>>
>>>>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>>>>> have to go with punctuate there too.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Michal
>>>>>>>
>>>>>>>
>>>>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>>>>
>>>>>>> Here's an example that we currently have.  We have a streams processor
>>>>>>> that does a transform from one topic into another. One of the fields in
>>>>>>> the source topic record is an expiration time, and one of the functions
>>>>>>> of the processor is to ensure that expired records get deleted promptly
>>>>>>> after that time passes (typically days or weeks after the message was
>>>>>>> originally produced). To do that, the processor keeps a state store of
>>>>>>> keys and expiration times, iterates that store in punctuate(), and
>>>>>>> emits delete (null) records for expired items. This needs to happen at
>>>>>>> some minimum interval regardless of the incoming message rate of the
>>>>>>> source topic.
>>>>>>>
>>>>>>> In this scenario, the expiration of records is the primary function of
>>>>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>>>>> range of use-cases.
>>>>>>>
>>>>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>>>>
>>>>>>> I apologize for the longer email below.  To my defense, it started
>>>>>>> out much
>>>>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>>>>> devil's
>>>>>>> advocate for a number of arguments brought forth in order to help
>>>>>>> improve
>>>>>>> this KIP -- I am not implying I necessarily disagree with the
>>>>>>> arguments.
>>>>>>>
>>>>>>> That aside, here are some further thoughts.
>>>>>>>
>>>>>>> First, there are (at least?) two categories for actions/behavior you
>>>>>>> invoke
>>>>>>> via punctuate():
>>>>>>>
>>>>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>>>>> to
>>>>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>>>>> the
>>>>>>> impact of punctuate is typically not observable by other processing
>>>>>>> nodes
>>>>>>> in the topology.
>>>>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>>>>> the
>>>>>>> punctuate is all about being observable by downstream processing
>>>>>>> nodes.
>>>>>>>
>>>>>>> A few releases back, we introduced record caches (DSL) and state
>>>>>>> store
>>>>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>>>>> relating to
>>>>>>> (2) where some users needed to control -- here: limit -- the
>>>>>>> downstream
>>>>>>> output rate of Kafka Streams because the downstream systems/apps
>>>>>>> would not
>>>>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>>>>> their
>>>>>>> scalability).  The argument for KIP-63, which notably did not
>>>>>>> introduce a
>>>>>>> "trigger" API, was that such an interaction with downstream systems
>>>>>>> is an
>>>>>>> operational concern;  it should not impact the processing *logic* of
>>>>>>> your
>>>>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>>>>> API,
>>>>>>> especially not the declarative DSL, with such operational concerns.
>>>>>>>
>>>>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>>>>> sorry, I
>>>>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>>>>> observing that our conversation is moving more and more into the
>>>>>>> direction
>>>>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>>>>> for
>>>>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>>>>> comments voiced here are about sth like "IF stream-time didn't
>>>>>>> trigger
>>>>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>>>>> want
>>>>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>>>>> related
>>>>>>> note, whatever we are discussing here will impact state store caches
>>>>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>>>>> should
>>>>>>> clarify any such impact here.
>>>>>>>
>>>>>>> Switching topics slightly.
>>>>>>>
>>>>>>> Jay wrote:
>>>>>>>
>>>>>>> One thing I've always found super important for this kind of design
>>>>>>> work
>>>>>>> is to do a really good job of cataloging the landscape of use cases
>>>>>>> and
>>>>>>> how prevalent each one is.
>>>>>>>
>>>>>>> +1 to this, as others have already said.
>>>>>>>
>>>>>>> Here, let me highlight -- just in case -- that when we talked about
>>>>>>> windowing use cases in the recent emails, the Processor API (where
>>>>>>> `punctuate` resides) does not have any notion of windowing at
>>>>>>> all.  If you
>>>>>>> want to do windowing *in the Processor API*, you must do so manually
>>>>>>> in
>>>>>>> combination with window stores.  For this reason I'd suggest to
>>>>>>> discuss use
>>>>>>> cases not just in general, but also in view of how you'd do so in the
>>>>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>>>>> `punctuate`
>>>>>>> does not impact the DSL at all, unless we add new functionality to
>>>>>>> it.
>>>>>>>
>>>>>>> Jay wrote in his strawman example:
>>>>>>>
>>>>>>> You aggregate click and impression data for a reddit like site.
>>>>>>> Every ten
>>>>>>> minutes you want to output a ranked list of the top 10 articles
>>>>>>> ranked by
>>>>>>> clicks/impressions for each geographical area. I want to be able
>>>>>>> run this
>>>>>>> in steady state as well as rerun to regenerate results (or catch up
>>>>>>> if it
>>>>>>> crashes).
>>>>>>>
>>>>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>>>>> we
>>>>>>> argued that the reason for saying "every ten minutes" above is not
>>>>>>> necessarily about because you want to output data *exactly* after ten
>>>>>>> minutes, but that you want to perform an aggregation based on 10-
>>>>>>> minute
>>>>>>> windows of input data; i.e., the point is about specifying the input
>>>>>>> for
>>>>>>> your aggregation, not or less about when the results of the
>>>>>>> aggregation
>>>>>>> should be send downstream.  To take an extreme example, you could
>>>>>>> disable
>>>>>>> record caches and let your app output a downstream update for every
>>>>>>> incoming input record.  If the last input record was from at minute 7
>>>>>>> of 10
>>>>>>> (for a 10-min window), then what your app would output at minute 10
>>>>>>> would
>>>>>>> be identical to what it had already emitted at minute 7 earlier
>>>>>>> anyways.
>>>>>>> This is particularly true when we take late-arriving data into
>>>>>>> account:  if
>>>>>>> a late record arrived at minute 13, your app would (by default) send
>>>>>>> a new
>>>>>>> update downstream, even though the "original" 10 minutes have already
>>>>>>> passed.
>>>>>>>
>>>>>>> Jay wrote...:
>>>>>>>
>>>>>>> There are a couple of tricky things that seem to make this hard
>>>>>>> with
>>>>>>>
>>>>>>> either
>>>>>>>
>>>>>>> of the options proposed:
>>>>>>> 1. If I emit this data using event time I have the problem
>>>>>>> described where
>>>>>>> a geographical region with no new clicks or impressions will fail
>>>>>>> to
>>>>>>>
>>>>>>> output
>>>>>>>
>>>>>>> results.
>>>>>>>
>>>>>>> ...and Arun Mathew wrote:
>>>>>>>
>>>>>>>
>>>>>>> We window by the event time, but trigger punctuate in <punctuate
>>>>>>> interval>
>>>>>>> duration of system time, in the absence of an event crossing the
>>>>>>> punctuate
>>>>>>> event time.
>>>>>>>
>>>>>>> So, given what I wrote above about the status quo and what you can
>>>>>>> already
>>>>>>> do with it, is the concern that the state store cache doesn't give
>>>>>>> you
>>>>>>> *direct* control over "forcing an output after no later than X
>>>>>>> seconds [of
>>>>>>> processing-time]" but only indirect control through a cache
>>>>>>> size?  (Note
>>>>>>> that I am not dismissing the claims why this might be helpful.)
>>>>>>>
>>>>>>> Arun Mathew wrote:
>>>>>>>
>>>>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>>>>> output the
>>>>>>> event counts on each topic on each cluster aggregated over a 1
>>>>>>> minute
>>>>>>> window. We have to use event time to be able to cross check the
>>>>>>> counts.
>>>>>>>
>>>>>>> But
>>>>>>>
>>>>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>>>>> time in
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>> absence of events. Otherwise the event counts for unexpired windows
>>>>>>> would
>>>>>>> be 0 which is bad.
>>>>>>>
>>>>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>>>>> absence
>>>>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>>>>> Scala
>>>>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>>>>> root
>>>>>>> cause that the downstream system interprets the absence of output in
>>>>>>> a
>>>>>>> particular way ("No output after 1 minute = I consider the output to
>>>>>>> be
>>>>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>>>>> possible)
>>>>>>> to correctly handle the difference between absence of output vs.
>>>>>>> output of
>>>>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>>>>> but
>>>>>>> want to understand the motivation better. :-)
>>>>>>>
>>>>>>> Also, to add some perspective, in some related discussions we talked
>>>>>>> about
>>>>>>> how a Kafka Streams application should not worry or not be coupled
>>>>>>> unnecessarily with such interpretation specifics in a downstream
>>>>>>> system's
>>>>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>>>>> more
>>>>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>>>>> than
>>>>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>>>>> and
>>>>>>> downstream systems, including helping to reconcile the differences in
>>>>>>> how
>>>>>>> these systems interpret changes, updates, late-arriving data,
>>>>>>> etc.  Kafka
>>>>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>>>>> processing from the specifics of downstream systems, thanks to
>>>>>>> specific
>>>>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>>>>> this
>>>>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>>>>> but
>>>>>>> it's currently awkward to use Connect for this", this might be a
>>>>>>> problem we
>>>>>>> can solve, too.)
>>>>>>>
>>>>>>> Switching topics slightly again.
>>>>>>>
>>>>>>> Thomas wrote:
>>>>>>>
>>>>>>> I'm not entirely convinced that a separate callback (option C)
>>>>>>> is that messy (it could just be a default method with an empty
>>>>>>> implementation), but if we wanted a single API to handle both
>>>>>>> cases,
>>>>>>> how about something like the following?
>>>>>>>
>>>>>>> enum Time {
>>>>>>>    STREAM,
>>>>>>>    CLOCK
>>>>>>> }
>>>>>>>
>>>>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>>>>> then
>>>>>>> whatever the user is doing inside this method is a black box to Kafka
>>>>>>> Streams (similar to how we have no idea what the user does inside a
>>>>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>>>>> won't
>>>>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>>>>> frequency than the processing-time action.  Or, we won't know whether
>>>>>>> the
>>>>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>>>>> minute of
>>>>>>> stream-time or 1-minute of processing-time, whichever comes
>>>>>>> first").  That
>>>>>>> said, I am not certain yet whether we would need such knowledge
>>>>>>> because,
>>>>>>> when using the Processor API, most of the work and decisions must be
>>>>>>> done
>>>>>>> by the user anyways.  It would matter though if the concept of
>>>>>>> "triggers"
>>>>>>> were to bubble up into the DSL because in the DSL the management of
>>>>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>>>>> Streams.
>>>>>>>
>>>>>>> [In any case, btw, we have the corner case where the user configured
>>>>>>> the
>>>>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>>>>> extractor), at which point both punctuate variants are based on the
>>>>>>> same
>>>>>>> time semantics / timeline.]
>>>>>>>
>>>>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>>>>> it
>>>>>>> this far. :-)
>>>>>>>
>>>>>>> More than happy to hear your thoughts!
>>>>>>> Michael
>>>>>>>
>>>>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>
>>>>>>> Thanks Matthias.
>>>>>>> Sure, will correct it right away.
>>>>>>>
>>>>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Thanks for preparing this page!
>>>>>>>
>>>>>>> About terminology:
>>>>>>>
>>>>>>> You introduce the term "event time" -- but we should call this
>>>>>>> "stream
>>>>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>>>>> this
>>>>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>>>>
>>>>>>> Does this make sense to you?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>>>
>>>>>>> Thanks Ewen.
>>>>>>>
>>>>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>>>>
>>>>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>>>
>>>>>>>
>>>>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>>>>> public.
>>>>>>>
>>>>>>> --
>>>>>>> Arun Mathew
>>>>>>>
>>>>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>>     Arun,
>>>>>>>
>>>>>>>     I've given you permission to edit the wiki. Let me know if
>>>>>>> you run
>>>>>>>
>>>>>>> into any
>>>>>>>
>>>>>>>     issues.
>>>>>>>
>>>>>>>     -Ewen
>>>>>>>
>>>>>>>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp> <am...@yahoo-corp.jp>
>>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>
>>>>>>>     > Thanks Michal. I don’t have the access yet [arunmathew88].
>>>>>>> Should I
>>>>>>>
>>>>>>> be
>>>>>>>
>>>>>>>     > sending a separate mail for this?
>>>>>>>     >
>>>>>>>     > I thought one of the person following this thread would be
>>>>>>> able to
>>>>>>>
>>>>>>> give me
>>>>>>>
>>>>>>>     > access.
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>>>>>>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>     > *Date: *Friday, April 7, 2017 at 17:16
>>>>>>>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>>>> semantics
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > Hi Arun,
>>>>>>>     >
>>>>>>>     > I was thinking along the same lines as you, listing the use
>>>>>>> cases
>>>>>>>
>>>>>>> on the
>>>>>>>
>>>>>>>     > wiki, but didn't find time to get around doing that yet.
>>>>>>>     > Don't mind if you do it if you have access now.
>>>>>>>     > I was thinking it would be nice if, once we have the use
>>>>>>> cases
>>>>>>>
>>>>>>> listed,
>>>>>>>
>>>>>>>     > people could use likes to up-vote the use cases similar to
>>>>>>> what
>>>>>>>
>>>>>>> they're
>>>>>>>
>>>>>>>     > working on.
>>>>>>>     >
>>>>>>>     > I should have a bit more time to action this in the next
>>>>>>> few days,
>>>>>>>
>>>>>>> but
>>>>>>>
>>>>>>>     > happy for you to do it if you can beat me to it ;-)
>>>>>>>     >
>>>>>>>     > Cheers,
>>>>>>>     > Michal
>>>>>>>     >
>>>>>>>     > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>>>     >
>>>>>>>     > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > Of course. I was thinking of a subpage where people can
>>>>>>>
>>>>>>> collaborate.
>>>>>>>
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > Will do as per Michael’s suggestion.
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > Regards,
>>>>>>>     >
>>>>>>>     > Arun Mathew
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>>> <
>>>>>>>
>>>>>>> matthias@confluent.io> wrote:
>>>>>>>
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >     Please share your Wiki-ID and a committer can give you
>>>>>>> write
>>>>>>>
>>>>>>> access.
>>>>>>>
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >     Btw: as you did not initiate the KIP, you should not
>>>>>>> change the
>>>>>>>
>>>>>>> KIP
>>>>>>>
>>>>>>>     >
>>>>>>>     >     without the permission of the original author -- in
>>>>>>> this case
>>>>>>>
>>>>>>> Michael.
>>>>>>>
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >     So you might also just share your thought over the
>>>>>>> mailing list
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>>     >
>>>>>>>     >     Michael can update the KIP page. Or, as an alternative,
>>>>>>> just
>>>>>>>
>>>>>>> create a
>>>>>>>
>>>>>>>     >
>>>>>>>     >     subpage for the KIP page.
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >     @Michael: WDYT?
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >     -Matthias
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>>>     >
>>>>>>>     >     > Hi Jay,
>>>>>>>     >
>>>>>>>     >     >           Thanks for the advise, I would like to list
>>>>>>> down
>>>>>>>
>>>>>>> the use cases as
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > per your suggestion. But it seems I don't have write
>>>>>>>
>>>>>>> permission to the
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > Apache Kafka Confluent Space. Whom shall I request
>>>>>>> for it?
>>>>>>>     >
>>>>>>>     >     >
>>>>>>>     >
>>>>>>>     >     > Regarding your last question. We are using a patch in
>>>>>>> our
>>>>>>>
>>>>>>> production system
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > which does exactly this.
>>>>>>>     >
>>>>>>>     >     > We window by the event time, but trigger punctuate in
>>>>>>>
>>>>>>> <punctuate interval>
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > duration of system time, in the absence of an event
>>>>>>> crossing
>>>>>>>
>>>>>>> the punctuate
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > event time.
>>>>>>>     >
>>>>>>>     >     >
>>>>>>>     >
>>>>>>>     >     > We are using Kafka Stream for our Audit Trail, where
>>>>>>> we need
>>>>>>>
>>>>>>> to output the
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > event counts on each topic on each cluster aggregated
>>>>>>> over a
>>>>>>>
>>>>>>> 1 minute
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > window. We have to use event time to be able to cross
>>>>>>> check
>>>>>>>
>>>>>>> the counts. But
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > we need to trigger punctuate [aggregate event pushes]
>>>>>>> by
>>>>>>>
>>>>>>> system time in the
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > absence of events. Otherwise the event counts for
>>>>>>> unexpired
>>>>>>>
>>>>>>> windows would
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > be 0 which is bad.
>>>>>>>     >
>>>>>>>     >     >
>>>>>>>     >
>>>>>>>     >     > "Maybe a hybrid solution works: I window by event
>>>>>>> time but
>>>>>>>
>>>>>>> trigger results
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > by system time for windows that have updated? Not
>>>>>>> really sure
>>>>>>>
>>>>>>> the details
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > of making that work. Does that work? Are there
>>>>>>> concrete
>>>>>>>
>>>>>>> examples where you
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > actually want the current behavior?"
>>>>>>>     >
>>>>>>>     >     >
>>>>>>>     >
>>>>>>>     >     > --
>>>>>>>     >
>>>>>>>     >     > With Regards,
>>>>>>>     >
>>>>>>>     >     >
>>>>>>>     >
>>>>>>>     >     > Arun Mathew
>>>>>>>     >
>>>>>>>     >     > Yahoo! JAPAN Corporation
>>>>>>>     >
>>>>>>>     >     >
>>>>>>>     >
>>>>>>>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>>>>
>>>>>>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >
>>>>>>>     >
>>>>>>>     >     >> Hi Jay,
>>>>>>>     >
>>>>>>>     >     >>
>>>>>>>     >
>>>>>>>     >     >> The hybrid solution is exactly what I expect and
>>>>>>> need for
>>>>>>>
>>>>>>> our use cases
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> when dealing with telecom data.
>>>>>>>     >
>>>>>>>     >     >>
>>>>>>>     >
>>>>>>>     >     >> Thanks
>>>>>>>     >
>>>>>>>     >     >> Tianji
>>>>>>>     >
>>>>>>>     >     >>
>>>>>>>     >
>>>>>>>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>>>>
>>>>>>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>
>>>>>>>     >
>>>>>>>     >     >>> Hey guys,
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>> One thing I've always found super important for
>>>>>>> this kind
>>>>>>>
>>>>>>> of design work
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> is
>>>>>>>     >
>>>>>>>     >     >>> to do a really good job of cataloging the landscape
>>>>>>> of use
>>>>>>>
>>>>>>> cases and how
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> prevalent each one is. By that I mean not just
>>>>>>> listing lots
>>>>>>>
>>>>>>> of uses, but
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> also grouping them into categories that
>>>>>>> functionally need
>>>>>>>
>>>>>>> the same thing.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> In the absence of this it is very hard to reason
>>>>>>> about
>>>>>>>
>>>>>>> design proposals.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> From the proposals so far I think we have a lot of
>>>>>>>
>>>>>>> discussion around
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> possible apis, but less around what the user needs
>>>>>>> for
>>>>>>>
>>>>>>> different use
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> cases
>>>>>>>     >
>>>>>>>     >     >>> and how they would implement that using the api.
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>> Here is an example:
>>>>>>>     >
>>>>>>>     >     >>> You aggregate click and impression data for a
>>>>>>> reddit like
>>>>>>>
>>>>>>> site. Every ten
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> minutes you want to output a ranked list of the top
>>>>>>> 10
>>>>>>>
>>>>>>> articles ranked by
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> clicks/impressions for each geographical area. I
>>>>>>> want to be
>>>>>>>
>>>>>>> able run this
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> in steady state as well as rerun to regenerate
>>>>>>> results (or
>>>>>>>
>>>>>>> catch up if it
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> crashes).
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>> There are a couple of tricky things that seem to
>>>>>>> make this
>>>>>>>
>>>>>>> hard with
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> either
>>>>>>>     >
>>>>>>>     >     >>> of the options proposed:
>>>>>>>     >
>>>>>>>     >     >>> 1. If I emit this data using event time I have the
>>>>>>> problem
>>>>>>>
>>>>>>> described
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> where
>>>>>>>     >
>>>>>>>     >     >>> a geographical region with no new clicks or
>>>>>>> impressions
>>>>>>>
>>>>>>> will fail to
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> output
>>>>>>>     >
>>>>>>>     >     >>> results.
>>>>>>>     >
>>>>>>>     >     >>> 2. If I emit this data using system time I have the
>>>>>>> problem
>>>>>>>
>>>>>>> that when
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> reprocessing data my window may not be ten minutes
>>>>>>> but 10
>>>>>>>
>>>>>>> hours if my
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> processing is very fast so it dramatically changes
>>>>>>> the
>>>>>>>
>>>>>>> output.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>> Maybe a hybrid solution works: I window by event
>>>>>>> time but
>>>>>>>
>>>>>>> trigger results
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> by system time for windows that have updated? Not
>>>>>>> really
>>>>>>>
>>>>>>> sure the details
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> of making that work. Does that work? Are there
>>>>>>> concrete
>>>>>>>
>>>>>>> examples where
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> you
>>>>>>>     >
>>>>>>>     >     >>> actually want the current behavior?
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>> -Jay
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>>>>
>>>>>>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> wrote:
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>>> Hi All,
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> Thanks for the KIP. We were also in need of a
>>>>>>> mechanism to
>>>>>>>
>>>>>>> trigger
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>> punctuate in the absence of events.
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> As I described in [
>>>>>>>     >
>>>>>>>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>>>     >
>>>>>>>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>>>     >
>>>>>>>     >     >>>> plugin.system.issuetabpanels:comment-
>>>>>>> tabpanel#comment-
>>>>>>>
>>>>>>> 15926036
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>> ],
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>>    - Our approached involved using the event time
>>>>>>> by
>>>>>>>
>>>>>>> default.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>    - The method to check if there is any punctuate
>>>>>>> ready
>>>>>>>
>>>>>>> in the
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>    PunctuationQueue is triggered via the any event
>>>>>>>
>>>>>>> received by the
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> stream
>>>>>>>     >
>>>>>>>     >     >>>>    tread, or at the polling intervals in the
>>>>>>> absence of
>>>>>>>
>>>>>>> any events.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>    - When we create Punctuate objects (which
>>>>>>> contains the
>>>>>>>
>>>>>>> next event
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> time
>>>>>>>     >
>>>>>>>     >     >>>>    for punctuation and interval), we also record
>>>>>>> the
>>>>>>>
>>>>>>> creation time
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> (system
>>>>>>>     >
>>>>>>>     >     >>>>    time).
>>>>>>>     >
>>>>>>>     >     >>>>    - While checking for maturity of Punctuate
>>>>>>> Schedule by
>>>>>>>     >
>>>>>>>     >     >> mayBePunctuate
>>>>>>>     >
>>>>>>>     >     >>>>    method, we also check if the system clock has
>>>>>>> elapsed
>>>>>>>
>>>>>>> the punctuate
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>    interval since the schedule creation time.
>>>>>>>     >
>>>>>>>     >     >>>>    - In the absence of any event, or in the
>>>>>>> absence of any
>>>>>>>
>>>>>>> event for
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> one
>>>>>>>     >
>>>>>>>     >     >>>>    topic in the partition group assigned to the
>>>>>>> stream
>>>>>>>
>>>>>>> task, the system
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>> time
>>>>>>>     >
>>>>>>>     >     >>>>    will elapse the interval and we trigger a
>>>>>>> punctuate
>>>>>>>
>>>>>>> using the
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> expected
>>>>>>>     >
>>>>>>>     >     >>>>    punctuation event time.
>>>>>>>     >
>>>>>>>     >     >>>>    - we then create the next punctuation schedule
>>>>>>> as
>>>>>>>
>>>>>>> punctuation event
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> time
>>>>>>>     >
>>>>>>>     >     >>>>    + punctuation interval, [again recording the
>>>>>>> system
>>>>>>>
>>>>>>> time of creation
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> of
>>>>>>>     >
>>>>>>>     >     >>>> the
>>>>>>>     >
>>>>>>>     >     >>>>    schedule].
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>>>> approach
>>>>>>>
>>>>>>> has pros and
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>> cons.
>>>>>>>     >
>>>>>>>     >     >>>> Pros
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>>    - Punctuates will happen in <punctuate
>>>>>>> interval> time
>>>>>>>
>>>>>>> duration at
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> max
>>>>>>>     >
>>>>>>>     >     >>> in
>>>>>>>     >
>>>>>>>     >     >>>>    terms of system time.
>>>>>>>     >
>>>>>>>     >     >>>>    - The semantics as a whole continues to revolve
>>>>>>> around
>>>>>>>
>>>>>>> event time.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>    - We can use the old data [old timestamps] to
>>>>>>> rerun any
>>>>>>>
>>>>>>> experiments
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> or
>>>>>>>     >
>>>>>>>     >     >>>>    tests.
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> Cons
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>>    - In case the  <punctuate interval> is not a
>>>>>>> time
>>>>>>>
>>>>>>> duration [say
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> logical
>>>>>>>     >
>>>>>>>     >     >>>>    time/event count], then the approach might not
>>>>>>> be
>>>>>>>
>>>>>>> meaningful.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>    - In case there is a case where we have to wait
>>>>>>> for an
>>>>>>>
>>>>>>> actual event
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> from
>>>>>>>     >
>>>>>>>     >     >>>>    a low event rate partition in the partition
>>>>>>> group, this
>>>>>>>
>>>>>>> approach
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> will
>>>>>>>     >
>>>>>>>     >     >>>> jump
>>>>>>>     >
>>>>>>>     >     >>>>    the gun.
>>>>>>>     >
>>>>>>>     >     >>>>    - in case the event processing cannot catch up
>>>>>>> with the
>>>>>>>
>>>>>>> event rate
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> and
>>>>>>>     >
>>>>>>>     >     >>>>    the expected timestamp events gets queued for
>>>>>>> long
>>>>>>>
>>>>>>> time, this
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> approach
>>>>>>>     >
>>>>>>>     >     >>>>    might jump the gun.
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> I believe the above approach and discussion goes
>>>>>>> close to
>>>>>>>
>>>>>>> the approach
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> A.
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> -----------
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> I like the idea of having an even count based
>>>>>>> punctuate.
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> -----------
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> I agree with the discussion around approach C,
>>>>>>> that we
>>>>>>>
>>>>>>> should provide
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> the
>>>>>>>     >
>>>>>>>     >     >>>> user with the option to choose system time or
>>>>>>> event time
>>>>>>>
>>>>>>> based
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> punctuates.
>>>>>>>     >
>>>>>>>     >     >>>> But I believe that the user predominantly wants to
>>>>>>> use
>>>>>>>
>>>>>>> event time while
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> not
>>>>>>>     >
>>>>>>>     >     >>>> missing out on regular punctuates due to event
>>>>>>> delays or
>>>>>>>
>>>>>>> event
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> absences.
>>>>>>>     >
>>>>>>>     >     >>>> Hence a complex punctuate option as Matthias
>>>>>>> mentioned
>>>>>>>
>>>>>>> (quoted below)
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> would
>>>>>>>     >
>>>>>>>     >     >>>> be most apt.
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> "- We might want to add "complex" schedules later
>>>>>>> on
>>>>>>>
>>>>>>> (like, punctuate
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> on
>>>>>>>     >
>>>>>>>     >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>>>> time
>>>>>>>
>>>>>>> whatever comes
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>> first)."
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> -----------
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> I think I read somewhere that Kafka Streams
>>>>>>> started with
>>>>>>>
>>>>>>> System Time as
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> the
>>>>>>>     >
>>>>>>>     >     >>>> punctuation standard, but was later changed to
>>>>>>> Event Time.
>>>>>>>
>>>>>>> I guess
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> there
>>>>>>>     >
>>>>>>>     >     >>>> would be some good reason behind it. As Kafka
>>>>>>> Streams want
>>>>>>>
>>>>>>> to evolve
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> more
>>>>>>>     >
>>>>>>>     >     >>>> on the Stream Processing front, I believe the
>>>>>>> emphasis on
>>>>>>>
>>>>>>> event time
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> would
>>>>>>>     >
>>>>>>>     >     >>>> remain quite strong.
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> With Regards,
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> Arun Mathew
>>>>>>>     >
>>>>>>>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>>>>
>>>>>>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> wrote:
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>>> Yeah I like PuncutationType much better; I just
>>>>>>> threw
>>>>>>>
>>>>>>> Time out there
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>>>> still
>>>>>>>
>>>>>>> think it's
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> worth considering what this buys us over an
>>>>>>> additional
>>>>>>>
>>>>>>> callback. I
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> foresee a number of punctuate implementations
>>>>>>> following
>>>>>>>
>>>>>>> this pattern:
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> public void punctuate(PunctuationType type) {
>>>>>>>     >
>>>>>>>     >     >>>>>     switch (type) {
>>>>>>>     >
>>>>>>>     >     >>>>>         case EVENT_TIME:
>>>>>>>     >
>>>>>>>     >     >>>>>             methodA();
>>>>>>>     >
>>>>>>>     >     >>>>>             break;
>>>>>>>     >
>>>>>>>     >     >>>>>         case SYSTEM_TIME:
>>>>>>>     >
>>>>>>>     >     >>>>>             methodB();
>>>>>>>     >
>>>>>>>     >     >>>>>             break;
>>>>>>>     >
>>>>>>>     >     >>>>>     }
>>>>>>>     >
>>>>>>>     >     >>>>> }
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> I guess one advantage of this approach is we
>>>>>>> could add
>>>>>>>
>>>>>>> additional
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> punctuation types later in a backwards compatible
>>>>>>> way
>>>>>>>
>>>>>>> (like event
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> count
>>>>>>>     >
>>>>>>>     >     >>>>> as you mentioned).
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> -Tommy
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>>>> Sax wrote:
>>>>>>>     >
>>>>>>>     >     >>>>>> That sounds promising.
>>>>>>>     >
>>>>>>>     >     >>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>> I am just wondering if `Time` is the best name.
>>>>>>> Maybe we
>>>>>>>
>>>>>>> want to
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> add
>>>>>>>     >
>>>>>>>     >     >>>>>> other non-time based punctuations at some point
>>>>>>> later. I
>>>>>>>
>>>>>>> would
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>> suggest
>>>>>>>     >
>>>>>>>     >     >>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>> enum PunctuationType {
>>>>>>>     >
>>>>>>>     >     >>>>>>   EVENT_TIME,
>>>>>>>     >
>>>>>>>     >     >>>>>>   SYSTEM_TIME,
>>>>>>>     >
>>>>>>>     >     >>>>>> }
>>>>>>>     >
>>>>>>>     >     >>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>> or similar. Just to keep the door open -- it's
>>>>>>> easier to
>>>>>>>
>>>>>>> add new
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>> stuff
>>>>>>>     >
>>>>>>>     >     >>>>>> if the name is more generic.
>>>>>>>     >
>>>>>>>     >     >>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>> -Matthias
>>>>>>>     >
>>>>>>>     >     >>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> I agree that the framework providing and
>>>>>>> managing the
>>>>>>>
>>>>>>> notion of
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> stream
>>>>>>>     >
>>>>>>>     >     >>>>>>> time is valuable and not something we would
>>>>>>> want to
>>>>>>>
>>>>>>> delegate to
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> the
>>>>>>>     >
>>>>>>>     >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>>>> separate
>>>>>>>
>>>>>>> callback
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> (option
>>>>>>>     >
>>>>>>>     >     >>>>>>> C)
>>>>>>>     >
>>>>>>>     >     >>>>>>> is that messy (it could just be a default
>>>>>>> method with
>>>>>>>
>>>>>>> an empty
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> implementation), but if we wanted a single API
>>>>>>> to
>>>>>>>
>>>>>>> handle both
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> cases,
>>>>>>>     >
>>>>>>>     >     >>>>>>> how about something like the following?
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> enum Time {
>>>>>>>     >
>>>>>>>     >     >>>>>>>    STREAM,
>>>>>>>     >
>>>>>>>     >     >>>>>>>    CLOCK
>>>>>>>     >
>>>>>>>     >     >>>>>>> }
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> Then on ProcessorContext:
>>>>>>>     >
>>>>>>>     >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>>>> We could
>>>>>>>
>>>>>>> allow
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> this
>>>>>>>     >
>>>>>>>     >     >>>>>>> to
>>>>>>>     >
>>>>>>>     >     >>>>>>> be called once for each value of time to mix
>>>>>>>
>>>>>>> approaches.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> Then the Processor API becomes:
>>>>>>>     >
>>>>>>>     >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>>>>
>>>>>>> schedule resulted
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> in
>>>>>>>     >
>>>>>>>     >     >>>>>>> this call.
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> Thoughts?
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>>>> Sax
>>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> I was thinking about the four options you
>>>>>>> proposed in
>>>>>>>
>>>>>>> more
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> details
>>>>>>>     >
>>>>>>>     >     >>>>>>>> and
>>>>>>>     >
>>>>>>>     >     >>>>>>>> this are my thoughts:
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> (A) You argue, that users can still
>>>>>>> "punctuate" on
>>>>>>>
>>>>>>> event-time
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> via
>>>>>>>     >
>>>>>>>     >     >>>>>>>> process(), but I am not sure if this is
>>>>>>> possible.
>>>>>>>
>>>>>>> Note, that
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> users
>>>>>>>     >
>>>>>>>     >     >>>>>>>> only
>>>>>>>     >
>>>>>>>     >     >>>>>>>> get record timestamps via context.timestamp().
>>>>>>> Thus,
>>>>>>>
>>>>>>> users
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> would
>>>>>>>     >
>>>>>>>     >     >>>>>>>> need
>>>>>>>     >
>>>>>>>     >     >>>>>>>> to
>>>>>>>     >
>>>>>>>     >     >>>>>>>> track the time progress per partition (based
>>>>>>> on the
>>>>>>>
>>>>>>> partitions
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> they
>>>>>>>     >
>>>>>>>     >     >>>>>>>> obverse via context.partition(). (This alone
>>>>>>> puts a
>>>>>>>
>>>>>>> huge burden
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> on
>>>>>>>     >
>>>>>>>     >     >>>>>>>> the
>>>>>>>     >
>>>>>>>     >     >>>>>>>> user by itself.) However, users are not
>>>>>>> notified at
>>>>>>>
>>>>>>> startup
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> what
>>>>>>>     >
>>>>>>>     >     >>>>>>>> partitions are assigned, and user are not
>>>>>>> notified
>>>>>>>
>>>>>>> when
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> partitions
>>>>>>>     >
>>>>>>>     >     >>>>>>>> get
>>>>>>>     >
>>>>>>>     >     >>>>>>>> revoked. Because this information is not
>>>>>>> available,
>>>>>>>
>>>>>>> it's not
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> possible
>>>>>>>     >
>>>>>>>     >     >>>>>>>> to
>>>>>>>     >
>>>>>>>     >     >>>>>>>> "manually advance" stream-time, and thus
>>>>>>> event-time
>>>>>>>
>>>>>>> punctuation
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> within
>>>>>>>     >
>>>>>>>     >     >>>>>>>> process() seems not to be possible -- or do
>>>>>>> you see a
>>>>>>>
>>>>>>> way to
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> get
>>>>>>>     >
>>>>>>>     >     >>>>>>>> it
>>>>>>>     >
>>>>>>>     >     >>>>>>>> done? And even if, it might still be too
>>>>>>> clumsy to
>>>>>>>
>>>>>>> use.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> (B) This does not allow to mix both
>>>>>>> approaches, thus
>>>>>>>
>>>>>>> limiting
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> what
>>>>>>>     >
>>>>>>>     >     >>>>>>>> users
>>>>>>>     >
>>>>>>>     >     >>>>>>>> can do.
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> (C) This should give all flexibility we need.
>>>>>>> However,
>>>>>>>
>>>>>>> just
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> adding
>>>>>>>     >
>>>>>>>     >     >>>>>>>> one
>>>>>>>     >
>>>>>>>     >     >>>>>>>> more method seems to be a solution that is too
>>>>>>> simple
>>>>>>>
>>>>>>> (cf my
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> comments
>>>>>>>     >
>>>>>>>     >     >>>>>>>> below).
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>>>> sure how
>>>>>>>
>>>>>>> a user
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> could
>>>>>>>     >
>>>>>>>     >     >>>>>>>> enable system-time and event-time punctuation
>>>>>>> in
>>>>>>>
>>>>>>> parallel.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> Overall options (C) seems to be the most
>>>>>>> promising
>>>>>>>
>>>>>>> approach to
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> me.
>>>>>>>     >
>>>>>>>     >     >>>>>>>> Because I also favor a clean API, we might
>>>>>>> keep
>>>>>>>
>>>>>>> current
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> punctuate()
>>>>>>>     >
>>>>>>>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>>>> at some
>>>>>>>
>>>>>>> later
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> point
>>>>>>>     >
>>>>>>>     >     >>>>>>>> when
>>>>>>>     >
>>>>>>>     >     >>>>>>>> people use the "new punctuate API".
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> Couple of follow up questions:
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> - I am wondering, if we should have two
>>>>>>> callback
>>>>>>>
>>>>>>> methods or
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> just
>>>>>>>     >
>>>>>>>     >     >>>>>>>> one
>>>>>>>     >
>>>>>>>     >     >>>>>>>> (ie, a unified for system and event time
>>>>>>> punctuation
>>>>>>>
>>>>>>> or one for
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> each?).
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> - If we have one, how can the user figure out,
>>>>>>> which
>>>>>>>
>>>>>>> condition
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> did
>>>>>>>     >
>>>>>>>     >     >>>>>>>> trigger?
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> - How would the API look like, for registering
>>>>>>>
>>>>>>> different
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> punctuate
>>>>>>>     >
>>>>>>>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> - We might want to add "complex" schedules
>>>>>>> later on
>>>>>>>
>>>>>>> (like,
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> punctuate
>>>>>>>     >
>>>>>>>     >     >>>>>>>> on
>>>>>>>     >
>>>>>>>     >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>>>> system-time
>>>>>>>
>>>>>>> whatever
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> comes
>>>>>>>     >
>>>>>>>     >     >>>>>>>> first). I don't say we should add this right
>>>>>>> away, but
>>>>>>>
>>>>>>> we might
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> want
>>>>>>>     >
>>>>>>>     >     >>>>>>>> to
>>>>>>>     >
>>>>>>>     >     >>>>>>>> define the API in a way, that it allows
>>>>>>> extensions
>>>>>>>
>>>>>>> like this
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> later
>>>>>>>     >
>>>>>>>     >     >>>>>>>> on,
>>>>>>>     >
>>>>>>>     >     >>>>>>>> without redesigning the API (ie, the API
>>>>>>> should be
>>>>>>>
>>>>>>> designed
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> extensible)
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> - Did you ever consider count-based
>>>>>>> punctuation?
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> I understand, that you would like to solve a
>>>>>>> simple
>>>>>>>
>>>>>>> problem,
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> but
>>>>>>>     >
>>>>>>>     >     >>>>>>>> we
>>>>>>>     >
>>>>>>>     >     >>>>>>>> learned from the past, that just "adding some
>>>>>>> API"
>>>>>>>
>>>>>>> quickly
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> leads
>>>>>>>     >
>>>>>>>     >     >>>>>>>> to a
>>>>>>>     >
>>>>>>>     >     >>>>>>>> not very well defined API that needs time
>>>>>>> consuming
>>>>>>>
>>>>>>> clean up
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> later on
>>>>>>>     >
>>>>>>>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>>>> holistic
>>>>>>>     >
>>>>>>>     >     >>>>>>>> punctuation
>>>>>>>     >
>>>>>>>     >     >>>>>>>> KIP
>>>>>>>     >
>>>>>>>     >     >>>>>>>> with this from the beginning on to avoid later
>>>>>>> painful
>>>>>>>     >
>>>>>>>     >     >> redesign.
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> -Matthias
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> Thanks Thomas,
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> I'm also wary of changing the existing
>>>>>>> semantics of
>>>>>>>     >
>>>>>>>     >     >> punctuate,
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> for
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> backward compatibility reasons, although I
>>>>>>> like the
>>>>>>>     >
>>>>>>>     >     >> conceptual
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> simplicity of that option.
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>>>> a way,
>>>>>>>
>>>>>>> uglier.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> I
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> added
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> this to the KIP now as option (C).
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>>>> more
>>>>>>>
>>>>>>> flexible,
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> as
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> it
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> allows
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> you to return any value, you're not limited
>>>>>>> to event
>>>>>>>
>>>>>>> time or
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> system
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> time
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> (although I don't see an actual use case
>>>>>>> where you
>>>>>>>
>>>>>>> might need
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> anything
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> else then those two). Hence I also proposed
>>>>>>> the
>>>>>>>
>>>>>>> option to
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> allow
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> users
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>>>> for
>>>>>>>
>>>>>>> them given
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> the
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> presence or absence of messages, much like
>>>>>>> they can
>>>>>>>
>>>>>>> decide
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> what
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> msg
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> time
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>>>> What do
>>>>>>>
>>>>>>> you
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> think
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> about
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> that? This is probably most flexible but also
>>>>>>> most
>>>>>>>     >
>>>>>>>     >     >> complicated.
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> All comments appreciated.
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> Cheers,
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> Michal
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> Although I fully agree we need a way to
>>>>>>> trigger
>>>>>>>
>>>>>>> periodic
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> processing
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> that is independent from whether and when
>>>>>>> messages
>>>>>>>
>>>>>>> arrive,
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> I'm
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> not sure
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> I like the idea of changing the existing
>>>>>>> semantics
>>>>>>>
>>>>>>> across
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> the
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> board.
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> What if we added an additional callback to
>>>>>>> Processor
>>>>>>>
>>>>>>> that
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> can
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> be
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>>>> always
>>>>>>>
>>>>>>> called at
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> fixed, wall
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>>>> have to
>>>>>>>
>>>>>>> give
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> up
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> the
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> notion
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> of stream time to be able to do periodic
>>>>>>> processing.
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>>>> Borowiecki
>>>>>>>
>>>>>>> wrote:
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> Hi all,
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>>>>
>>>>>>> punctuate
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> semantics
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>>>>
>>>>>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/ confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>>>
>>>>>>>     >
>>>>>>>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>>>>
>>>>>>> 138%
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> 3A+C
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> hange+
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> punctuate+semantics>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> .
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> Appreciating there can be different views
>>>>>>> on
>>>>>>>
>>>>>>> system-time
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> vs
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> event-
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> time
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>>>> case and
>>>>>>>
>>>>>>> the
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> importance of
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>>>> I've
>>>>>>>
>>>>>>> left it
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> quite
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> open
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> and
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>>>>
>>>>>>> progresses.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> Thanks,
>>>>>>>     >
>>>>>>>     >     >>>>>>>>>>> Michal
>>>>>>>     >
>>>>>>>     >     >>>>>>> --
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>     Tommy Becker
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>     Senior Software Engineer
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>     tivo.com
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> ________________________________
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> This email and any attachments may contain
>>>>>>> confidential
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> privileged material for the sole use of the
>>>>>>> intended
>>>>>>>
>>>>>>> recipient.
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> Any
>>>>>>>     >
>>>>>>>     >     >>>>>>> review, copying, or distribution of this email
>>>>>>> (or any
>>>>>>>     >
>>>>>>>     >     >> attachments)
>>>>>>>     >
>>>>>>>     >     >>>>>>> by others is prohibited. If you are not the
>>>>>>> intended
>>>>>>>
>>>>>>> recipient,
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> please contact the sender immediately and
>>>>>>> permanently
>>>>>>>
>>>>>>> delete this
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> email and any attachments. No employee or agent
>>>>>>> of TiVo
>>>>>>>
>>>>>>> Inc. is
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> authorized to conclude any binding agreement on
>>>>>>> behalf
>>>>>>>
>>>>>>> of TiVo
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> Inc.
>>>>>>>     >
>>>>>>>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>>>> only be
>>>>>>>
>>>>>>> made by a
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>>> signed written agreement.
>>>>>>>     >
>>>>>>>     >     >>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> --
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>     Tommy Becker
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>     Senior Software Engineer
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>     tivo.com
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> ________________________________
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> This email and any attachments may contain
>>>>>>> confidential
>>>>>>>
>>>>>>> and
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> privileged
>>>>>>>     >
>>>>>>>     >     >>>>> material for the sole use of the intended
>>>>>>> recipient. Any
>>>>>>>
>>>>>>> review,
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>> copying,
>>>>>>>     >
>>>>>>>     >     >>>>> or distribution of this email (or any
>>>>>>> attachments) by
>>>>>>>
>>>>>>> others is
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>> prohibited.
>>>>>>>     >
>>>>>>>     >     >>>>> If you are not the intended recipient, please
>>>>>>> contact the
>>>>>>>
>>>>>>> sender
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> immediately and permanently delete this email and
>>>>>>> any
>>>>>>>
>>>>>>> attachments. No
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>>>> conclude
>>>>>>>
>>>>>>> any binding
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>>>> Binding
>>>>>>>
>>>>>>> agreements with
>>>>>>>
>>>>>>>     >
>>>>>>>     >     >> TiVo
>>>>>>>     >
>>>>>>>     >     >>>>> Inc. may only be made by a signed written
>>>>>>> agreement.
>>>>>>>     >
>>>>>>>     >     >>>>>
>>>>>>>     >
>>>>>>>     >     >>>>
>>>>>>>     >
>>>>>>>     >     >>>
>>>>>>>     >
>>>>>>>     >     >>
>>>>>>>     >
>>>>>>>     >     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > --
>>>>>>>     >
>>>>>>>     > <http://www.openbet.com/> <http://www.openbet.com/>
>>>>>>>
>>>>>>>     >
>>>>>>>     > *Michal Borowiecki*
>>>>>>>     >
>>>>>>>     > *Senior Software Engineer L4*
>>>>>>>     >
>>>>>>>     > *T: *
>>>>>>>     >
>>>>>>>     > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>>>>     >
>>>>>>>     > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > *E: *
>>>>>>>     >
>>>>>>>     > michal.borowiecki@openbet.com
>>>>>>>     >
>>>>>>>     > *W: *
>>>>>>>     >
>>>>>>>     > www.openbet.com
>>>>>>>     >
>>>>>>>     > *OpenBet Ltd*
>>>>>>>     >
>>>>>>>     > Chiswick Park Building 9
>>>>>>>     >
>>>>>>>     > 566 Chiswick High Rd
>>>>>>>     >
>>>>>>>     > London
>>>>>>>     >
>>>>>>>     > W4 5XT
>>>>>>>     >
>>>>>>>     > UK
>>>>>>>     >
>>>>>>>     > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>     > This message is confidential and intended only for the
>>>>>>> addressee.
>>>>>>>
>>>>>>> If you
>>>>>>>
>>>>>>>     > have received this message in error, please immediately
>>>>>>> notify the
>>>>>>>     > postmaster@openbet.com and delete it from your system as
>>>>>>> well as
>>>>>>>
>>>>>>> any
>>>>>>>
>>>>>>>     > copies. The content of e-mails as well as traffic data may
>>>>>>> be
>>>>>>>
>>>>>>> monitored by
>>>>>>>
>>>>>>>     > OpenBet for employment and security purposes. To protect
>>>>>>> the
>>>>>>>
>>>>>>> environment
>>>>>>>
>>>>>>>     > please do not print this e-mail unless necessary. OpenBet
>>>>>>> Ltd.
>>>>>>>
>>>>>>> Registered
>>>>>>>
>>>>>>>     > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>>>> London,
>>>>>>>
>>>>>>> W4 5XT,
>>>>>>>
>>>>>>>     > United Kingdom. A company registered in England and Wales.
>>>>>>>
>>>>>>> Registered no.
>>>>>>>
>>>>>>>     > 3134634. VAT no. GB927523612
>>>>>>>     >
>>>>>>>     >
>>>>>>>     >
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>>
>>>>>>>
>>>>>>>     Tommy Becker
>>>>>>>
>>>>>>>     Senior Software Engineer
>>>>>>>
>>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747>
>>>>>>>
>>>>>>>
>>>>>>>     tivo.com
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> ________________________________
>>>>>>>
>>>>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> <http://www.openbet.com/> Michal Borowiecki
>>>>>>> Senior Software Engineer L4
>>>>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>>>>
>>>>>>>
>>>>>>>
>>>> -- 
>>>> Signature
>>>> <http://www.openbet.com/> 	Michal Borowiecki
>>>> Senior Software Engineer L4
>>>> 	T: 	+44 208 742 1600
>>>>
>>>> 	
>>>> 	+44 203 249 8448
>>>>
>>>> 	
>>>> 	 
>>>> 	E: 	michal.borowiecki@openbet.com
>>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>>
>>>> 	
>>>> 	OpenBet Ltd
>>>>
>>>> 	Chiswick Park Building 9
>>>>
>>>> 	566 Chiswick High Rd
>>>>
>>>> 	London
>>>>
>>>> 	W4 5XT
>>>>
>>>> 	UK
>>>>
>>>> 	
>>>> <https://www.openbet.com/email_promo>
>>>>
>>>> This message is confidential and intended only for the addressee. If
>>>> you have received this message in error, please immediately notify the
>>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>>> from your system as well as any copies. The content of e-mails as well
>>>> as traffic data may be monitored by OpenBet for employment and
>>>> security purposes. To protect the environment please do not print this
>>>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>>> company registered in England and Wales. Registered no. 3134634. VAT
>>>> no. GB927523612
>>>>
>>> -- 
>>> Signature
>>> <http://www.openbet.com/> 	Michal Borowiecki
>>> Senior Software Engineer L4
>>> 	T: 	+44 208 742 1600
>>>
>>> 	
>>> 	+44 203 249 8448
>>>
>>> 	
>>> 	 
>>> 	E: 	michal.borowiecki@openbet.com
>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>
>>> 	
>>> 	OpenBet Ltd
>>>
>>> 	Chiswick Park Building 9
>>>
>>> 	566 Chiswick High Rd
>>>
>>> 	London
>>>
>>> 	W4 5XT
>>>
>>> 	UK
>>>
>>> 	
>>> <https://www.openbet.com/email_promo>
>>>
>>> This message is confidential and intended only for the addressee. If you
>>> have received this message in error, please immediately notify the
>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>> from your system as well as any copies. The content of e-mails as well
>>> as traffic data may be monitored by OpenBet for employment and security
>>> purposes. To protect the environment please do not print this e-mail
>>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>>> registered in England and Wales. Registered no. 3134634. VAT no.
>>> GB927523612
>>>
> 
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
> 
> 	
> 	+44 203 249 8448
> 
> 	
> 	 
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
> 
> 	
> 	OpenBet Ltd
> 
> 	Chiswick Park Building 9
> 
> 	566 Chiswick High Rd
> 
> 	London
> 
> 	W4 5XT
> 
> 	UK
> 
> 	
> <https://www.openbet.com/email_promo>
> 
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com <ma...@openbet.com> and delete it
> from your system as well as any copies. The content of e-mails as well
> as traffic data may be monitored by OpenBet for employment and security
> purposes. To protect the environment please do not print this e-mail
> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> registered in England and Wales. Registered no. 3134634. VAT no.
> GB927523612
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi Matthias,

I agree it's difficult to reason about the hybrid approach, I certainly 
found it hard and I'm totally on board with the mantra.

I'd be happy to limit the scope of this KIP to add system-time 
punctuation semantics (in addition to existing stream-time semantics) 
and leave more complex schemes for users to implement on top of that.

Further additional PunctuationTypes, could then be added by future KIPs, 
possibly including the hybrid approach once it has been given more thought.

> There are real-time applications, that want to get
> callbacks in regular system-time intervals (completely independent from
> stream-time).
Can you please describe what they are, so that I can put them on the 
wiki for later reference?

Thanks,

Michal


On 23/04/17 21:27, Matthias J. Sax wrote:
> Hi,
>
> I do like Damian's API proposal about the punctuation callback function.
>
> I also did reread the KIP and thought about the semantics we want to
> provide.
>
>> Given the above, I don't see a reason any more for a separate system-time based punctuation.
> I disagree here. There are real-time applications, that want to get
> callbacks in regular system-time intervals (completely independent from
> stream-time). Thus we should allow this -- if we really follow the
> "hybrid" approach, this could be configured with stream-time interval
> infinite and delay whatever system-time punctuation interval you want to
> have. However, I would like to add a proper API for this and do this
> configuration under the hood (that would allow one implementation within
> all kind of branching for different cases).
>
> Thus, we definitely should have PunctutionType#StreamTime and
> #SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
> a fan of your latest API proposal.
>
>
> About the hybrid approach in general. On the one hand I like it, on the
> other hand, it seems to be rather (1) complicated (not necessarily from
> an implementation point of view, but for people to understand it) and
> (2) mixes two semantics together in a "weird" way". Thus, I disagree with:
>
>> It may appear complicated at first but I do think these semantics will
>> still be more understandable to users than having 2 separate punctuation
>> schedules/callbacks with different PunctuationTypes.
> This statement only holds if you apply strong assumptions that I don't
> believe hold in general -- see (2) for details -- and I think it is
> harder than you assume to reason about the hybrid approach in general.
> IMHO, the hybrid approach is a "false friend" that seems to be easy to
> reason about...
>
>
> (1) Streams always embraced "easy to use" and we should really be
> careful to keep it this way. On the other hand, as we are talking about
> changes to PAPI, it won't affect DSL users (DSL does not use punctuation
> at all at the moment), and thus, the "easy to use" mantra might not be
> affected, while it will allow advanced users to express more complex stuff.
>
> I like the mantra: "make simple thing easy and complex things possible".
>
> (2) IMHO the major disadvantage (issue?) of the hybrid approach is the
> implicit assumption that even-time progresses at the same "speed" as
> system-time during regular processing. This implies the assumption that
> a slower progress in stream-time indicates the absence of input events
> (and that later arriving input events will have a larger event-time with
> high probability). Even if this might be true for some use cases, I
> doubt it holds in general. Assume that you get a spike in traffic and
> for some reason stream-time does advance slowly because you have more
> records to process. This might trigger a system-time based punctuation
> call even if this seems not to be intended. I strongly believe that it
> is not easy to reason about the semantics of the hybrid approach (even
> if the intentional semantics would be super useful -- but I doubt that
> we get want we ask for).
>
> Thus, I also believe that one might need different "configuration"
> values for the hybrid approach if you run the same code for different
> scenarios: regular processing, re-processing, catching up scenario. And
> as the term "configuration" implies, we might be better off to not mix
> configuration with business logic that is expressed via code.
>
>
> One more comment: I also don't think that the hybrid approach is
> deterministic as claimed in the use-case subpage. I understand the
> reasoning and agree, that it is deterministic if certain assumptions
> hold -- compare above -- and if configured correctly. But strictly
> speaking it's not because there is a dependency on system-time (and
> IMHO, if system-time is involved it cannot be deterministic by definition).
>
>
>> I see how in theory this could be implemented on top of the 2 punctuate
>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>> the other system-time based) but it would be a much more complicated
>> scheme and I don't want to suggest that.
> I agree that expressing the intended hybrid semantics is harder if we
> offer only #StreamTime and #SystemTime punctuation. However, I also
> believe that the hybrid approach is a "false friend" with regard to
> reasoning about the semantics (it indicates that it more easy as it is
> in reality). Therefore, we might be better off to not offer the hybrid
> approach and make it clear to a developed, that it is hard to mix
> #StreamTime and #SystemTime in a semantically sound way.
>
>
> Looking forward to your feedback. :)
>
> -Matthias
>
>
>
>
> On 4/22/17 11:43 AM, Michal Borowiecki wrote:
>> Hi all,
>>
>> Looking for feedback on the functional interface approach Damian
>> proposed. What do people think?
>>
>> Further on the semantics of triggering punctuate though:
>>
>> I ran through the 2 use cases that Arun had kindly put on the wiki
>> (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
>> in my head and on a whiteboard and I can't find a better solution than
>> the "hybrid" approach he had proposed.
>>
>> I see how in theory this could be implemented on top of the 2 punctuate
>> callbacks with the 2 different PunctuationTypes (one stream-time based,
>> the other system-time based) but it would be a much more complicated
>> scheme and I don't want to suggest that.
>>
>> However, to add to the hybrid algorithm proposed, I suggest one
>> parameter to that: a tolerance period, expressed in milliseconds
>> system-time, after which the punctuation will be invoked in case the
>> stream-time advance hasn't triggered it within the requested interval
>> since the last invocation of punctuate (as recorded in system-time)
>>
>> This would allow a user-defined tolerance for late arriving events. The
>> trade off would be left for the user to decide: regular punctuation in
>> the case of absence of events vs allowing for records arriving late or
>> some build-up due to processing not catching up with the event rate.
>> In the one extreme, this tolerance could be set to infinity, turning
>> hybrid into simply stream-time based punctuate, like we have now. In the
>> other extreme, the tolerance could be set to 0, resulting in a
>> system-time upper bound on the effective punctuation interval.
>>
>> Given the above, I don't see a reason any more for a separate
>> system-time based punctuation. The "hybrid" approach with 0ms tolerance
>> would under normal operation trigger at regular intervals wrt the
>> system-time, except in cases of re-play/catch-up, where the stream time
>> advances faster than system time. In these cases punctuate would happen
>> more often than the specified interval wrt system time. However, the
>> use-cases that need system-time punctuations (that I've seen at least)
>> really only have a need for an upper bound on punctuation delay but
>> don't need a lower bound.
>>
>> To that effect I'd propose the api to be as follows, on ProcessorContext:
>>
>> schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
>>
>> schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
>>
>> Punctuation is triggered when either:
>> - the stream time advances past the (stream time of the previous
>> punctuation) + interval;
>> - or (iff the toleranceInterval is set) when the system time advances
>> past the (system time of the previous punctuation) + interval +
>> toleranceInterval
>>
>> In either case:
>> - we trigger punctuate passing as the argument the stream time at which
>> the current punctuation was meant to happen
>> - next punctuate is scheduled at (stream time at which the current
>> punctuation was meant to happen) + interval
>>
>> It may appear complicated at first but I do think these semantics will
>> still be more understandable to users than having 2 separate punctuation
>> schedules/callbacks with different PunctuationTypes.
>>
>>
>>
>> PS. Having re-read this, maybe the following alternative would be easier
>> to understand (WDYT?):
>>
>> schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
>>
>> schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
>>
>> Punctuation is triggered when either:
>> - the stream time advances past the (stream time of the previous
>> punctuation) + streamTimeInterval;
>> - or (iff systemTimeUpperBound is set) when the system time advances
>> past the (system time of the previous punctuation) + systemTimeUpperBound
>>
>> Awaiting comments.
>>
>> Thanks,
>> Michal
>>
>> On 21/04/17 16:56, Michal Borowiecki wrote:
>>> Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>>> in favour of something that doesn't return a record. Not a problem though.
>>>
>>>
>>> On 21/04/17 16:32, Damian Guy wrote:
>>>> Thanks Michal,
>>>> I agree Transformer.punctuate should also be void, but we can deprecate
>>>> that too in favor of the new interface.
>>>>
>>>> Thanks for the javadoc PR!
>>>>
>>>> Cheers,
>>>> Damian
>>>>
>>>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>>> michal.borowiecki@openbet.com> wrote:
>>>>
>>>>> Yes, that looks better to me.
>>>>>
>>>>> Note that punctuate on Transformer is currently returning a record, but I
>>>>> think it's ok to have all output records be sent via
>>>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>>>> multiple records from one invocation of punctuate.
>>>>>
>>>>> This way it's consistent between Processor and Transformer.
>>>>>
>>>>>
>>>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>>>> there:
>>>>>
>>>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>>>
>>>>> and PR: https://github.com/apache/kafka/pull/2884
>>>>>
>>>>> Cheers,
>>>>>
>>>>> Michal
>>>>> On 20/04/17 18:55, Damian Guy wrote:
>>>>>
>>>>> Hi Michal,
>>>>>
>>>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>>>> the API.
>>>>> 1. deprecate the punctuate method on Processor
>>>>> 2. create a new Functional Interface just for Punctuation, something like:
>>>>> interface Punctuator {
>>>>>      void punctuate(long timestamp)
>>>>> }
>>>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>>>> interval, PunctuationType type, Punctuator callback)
>>>>> 4. deprecate the existing schedule function
>>>>>
>>>>> Thoughts?
>>>>>
>>>>> Thanks,
>>>>> Damian
>>>>>
>>>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>>>> michal.borowiecki@openbet.com> wrote:
>>>>>
>>>>>> Hi Thomas,
>>>>>>
>>>>>> I would say our use cases fall in the same category as yours.
>>>>>>
>>>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>>>
>>>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>>>> of design:
>>>>>>
>>>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>>>
>>>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>>>
>>>>>> Since the output field X is derived in some non-trivial way, we don't
>>>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>>>> another input field Z).
>>>>>>
>>>>>> So we have kv stores with the records and an additional kv store with
>>>>>> timestamp->id mapping which act like an index where we periodically do a
>>>>>> ranged query.
>>>>>>
>>>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>>>> work when there were no regular msgs on the input topic.
>>>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>>>> outside we created a "ticker" that produced msgs once per second onto
>>>>>> another topic and fed it into the same topology to trigger punctuate.
>>>>>> This didn't work either, which was much more surprising to us at the
>>>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>>>> *all* input partitions receive messages regularly.
>>>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>>>> is consumed by the second topology and is its only input topic. There's a
>>>>>> transformer on that topic which populates and updates the time-based
>>>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>>>> elapsed, the record id is sent to the main transformer, which
>>>>>> updates/deletes the record from the main kv store and forwards the
>>>>>> transformed record to the output topic.
>>>>>>
>>>>>> To me this setup feels horrendously complicated for what it does.
>>>>>>
>>>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>>>> However, the ticks don't feel so hacky when you realise they give you
>>>>>> some hypothetical benefits in predictability. You can reprocess the
>>>>>> messages in a reproducible manner, since the topologies use event-time,
>>>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>>>> reprocessing).
>>>>>> To make that work though, we would have to have the stream time advance
>>>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>>>> presence of messages on the other input topic.
>>>>>>
>>>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>>>> and the hybrid would work to simplify this a lot.
>>>>>>
>>>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>>>> have to go with punctuate there too.
>>>>>>
>>>>>> Thanks,
>>>>>> Michal
>>>>>>
>>>>>>
>>>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>>>
>>>>>> Here's an example that we currently have.  We have a streams processor
>>>>>> that does a transform from one topic into another. One of the fields in
>>>>>> the source topic record is an expiration time, and one of the functions
>>>>>> of the processor is to ensure that expired records get deleted promptly
>>>>>> after that time passes (typically days or weeks after the message was
>>>>>> originally produced). To do that, the processor keeps a state store of
>>>>>> keys and expiration times, iterates that store in punctuate(), and
>>>>>> emits delete (null) records for expired items. This needs to happen at
>>>>>> some minimum interval regardless of the incoming message rate of the
>>>>>> source topic.
>>>>>>
>>>>>> In this scenario, the expiration of records is the primary function of
>>>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>>>> range of use-cases.
>>>>>>
>>>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>>>
>>>>>> I apologize for the longer email below.  To my defense, it started
>>>>>> out much
>>>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>>>> devil's
>>>>>> advocate for a number of arguments brought forth in order to help
>>>>>> improve
>>>>>> this KIP -- I am not implying I necessarily disagree with the
>>>>>> arguments.
>>>>>>
>>>>>> That aside, here are some further thoughts.
>>>>>>
>>>>>> First, there are (at least?) two categories for actions/behavior you
>>>>>> invoke
>>>>>> via punctuate():
>>>>>>
>>>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>>>> to
>>>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>>>> the
>>>>>> impact of punctuate is typically not observable by other processing
>>>>>> nodes
>>>>>> in the topology.
>>>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>>>> the
>>>>>> punctuate is all about being observable by downstream processing
>>>>>> nodes.
>>>>>>
>>>>>> A few releases back, we introduced record caches (DSL) and state
>>>>>> store
>>>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>>>> relating to
>>>>>> (2) where some users needed to control -- here: limit -- the
>>>>>> downstream
>>>>>> output rate of Kafka Streams because the downstream systems/apps
>>>>>> would not
>>>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>>>> their
>>>>>> scalability).  The argument for KIP-63, which notably did not
>>>>>> introduce a
>>>>>> "trigger" API, was that such an interaction with downstream systems
>>>>>> is an
>>>>>> operational concern;  it should not impact the processing *logic* of
>>>>>> your
>>>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>>>> API,
>>>>>> especially not the declarative DSL, with such operational concerns.
>>>>>>
>>>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>>>> sorry, I
>>>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>>>> observing that our conversation is moving more and more into the
>>>>>> direction
>>>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>>>> for
>>>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>>>> comments voiced here are about sth like "IF stream-time didn't
>>>>>> trigger
>>>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>>>> want
>>>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>>>> related
>>>>>> note, whatever we are discussing here will impact state store caches
>>>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>>>> should
>>>>>> clarify any such impact here.
>>>>>>
>>>>>> Switching topics slightly.
>>>>>>
>>>>>> Jay wrote:
>>>>>>
>>>>>> One thing I've always found super important for this kind of design
>>>>>> work
>>>>>> is to do a really good job of cataloging the landscape of use cases
>>>>>> and
>>>>>> how prevalent each one is.
>>>>>>
>>>>>> +1 to this, as others have already said.
>>>>>>
>>>>>> Here, let me highlight -- just in case -- that when we talked about
>>>>>> windowing use cases in the recent emails, the Processor API (where
>>>>>> `punctuate` resides) does not have any notion of windowing at
>>>>>> all.  If you
>>>>>> want to do windowing *in the Processor API*, you must do so manually
>>>>>> in
>>>>>> combination with window stores.  For this reason I'd suggest to
>>>>>> discuss use
>>>>>> cases not just in general, but also in view of how you'd do so in the
>>>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>>>> `punctuate`
>>>>>> does not impact the DSL at all, unless we add new functionality to
>>>>>> it.
>>>>>>
>>>>>> Jay wrote in his strawman example:
>>>>>>
>>>>>> You aggregate click and impression data for a reddit like site.
>>>>>> Every ten
>>>>>> minutes you want to output a ranked list of the top 10 articles
>>>>>> ranked by
>>>>>> clicks/impressions for each geographical area. I want to be able
>>>>>> run this
>>>>>> in steady state as well as rerun to regenerate results (or catch up
>>>>>> if it
>>>>>> crashes).
>>>>>>
>>>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>>>> we
>>>>>> argued that the reason for saying "every ten minutes" above is not
>>>>>> necessarily about because you want to output data *exactly* after ten
>>>>>> minutes, but that you want to perform an aggregation based on 10-
>>>>>> minute
>>>>>> windows of input data; i.e., the point is about specifying the input
>>>>>> for
>>>>>> your aggregation, not or less about when the results of the
>>>>>> aggregation
>>>>>> should be send downstream.  To take an extreme example, you could
>>>>>> disable
>>>>>> record caches and let your app output a downstream update for every
>>>>>> incoming input record.  If the last input record was from at minute 7
>>>>>> of 10
>>>>>> (for a 10-min window), then what your app would output at minute 10
>>>>>> would
>>>>>> be identical to what it had already emitted at minute 7 earlier
>>>>>> anyways.
>>>>>> This is particularly true when we take late-arriving data into
>>>>>> account:  if
>>>>>> a late record arrived at minute 13, your app would (by default) send
>>>>>> a new
>>>>>> update downstream, even though the "original" 10 minutes have already
>>>>>> passed.
>>>>>>
>>>>>> Jay wrote...:
>>>>>>
>>>>>> There are a couple of tricky things that seem to make this hard
>>>>>> with
>>>>>>
>>>>>> either
>>>>>>
>>>>>> of the options proposed:
>>>>>> 1. If I emit this data using event time I have the problem
>>>>>> described where
>>>>>> a geographical region with no new clicks or impressions will fail
>>>>>> to
>>>>>>
>>>>>> output
>>>>>>
>>>>>> results.
>>>>>>
>>>>>> ...and Arun Mathew wrote:
>>>>>>
>>>>>>
>>>>>> We window by the event time, but trigger punctuate in <punctuate
>>>>>> interval>
>>>>>> duration of system time, in the absence of an event crossing the
>>>>>> punctuate
>>>>>> event time.
>>>>>>
>>>>>> So, given what I wrote above about the status quo and what you can
>>>>>> already
>>>>>> do with it, is the concern that the state store cache doesn't give
>>>>>> you
>>>>>> *direct* control over "forcing an output after no later than X
>>>>>> seconds [of
>>>>>> processing-time]" but only indirect control through a cache
>>>>>> size?  (Note
>>>>>> that I am not dismissing the claims why this might be helpful.)
>>>>>>
>>>>>> Arun Mathew wrote:
>>>>>>
>>>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>>>> output the
>>>>>> event counts on each topic on each cluster aggregated over a 1
>>>>>> minute
>>>>>> window. We have to use event time to be able to cross check the
>>>>>> counts.
>>>>>>
>>>>>> But
>>>>>>
>>>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>>>> time in
>>>>>>
>>>>>> the
>>>>>>
>>>>>> absence of events. Otherwise the event counts for unexpired windows
>>>>>> would
>>>>>> be 0 which is bad.
>>>>>>
>>>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>>>> absence
>>>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>>>> Scala
>>>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>>>> root
>>>>>> cause that the downstream system interprets the absence of output in
>>>>>> a
>>>>>> particular way ("No output after 1 minute = I consider the output to
>>>>>> be
>>>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>>>> possible)
>>>>>> to correctly handle the difference between absence of output vs.
>>>>>> output of
>>>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>>>> but
>>>>>> want to understand the motivation better. :-)
>>>>>>
>>>>>> Also, to add some perspective, in some related discussions we talked
>>>>>> about
>>>>>> how a Kafka Streams application should not worry or not be coupled
>>>>>> unnecessarily with such interpretation specifics in a downstream
>>>>>> system's
>>>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>>>> more
>>>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>>>> than
>>>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>>>> and
>>>>>> downstream systems, including helping to reconcile the differences in
>>>>>> how
>>>>>> these systems interpret changes, updates, late-arriving data,
>>>>>> etc.  Kafka
>>>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>>>> processing from the specifics of downstream systems, thanks to
>>>>>> specific
>>>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>>>> this
>>>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>>>> but
>>>>>> it's currently awkward to use Connect for this", this might be a
>>>>>> problem we
>>>>>> can solve, too.)
>>>>>>
>>>>>> Switching topics slightly again.
>>>>>>
>>>>>> Thomas wrote:
>>>>>>
>>>>>> I'm not entirely convinced that a separate callback (option C)
>>>>>> is that messy (it could just be a default method with an empty
>>>>>> implementation), but if we wanted a single API to handle both
>>>>>> cases,
>>>>>> how about something like the following?
>>>>>>
>>>>>> enum Time {
>>>>>>     STREAM,
>>>>>>     CLOCK
>>>>>> }
>>>>>>
>>>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>>>> then
>>>>>> whatever the user is doing inside this method is a black box to Kafka
>>>>>> Streams (similar to how we have no idea what the user does inside a
>>>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>>>> won't
>>>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>>>> frequency than the processing-time action.  Or, we won't know whether
>>>>>> the
>>>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>>>> minute of
>>>>>> stream-time or 1-minute of processing-time, whichever comes
>>>>>> first").  That
>>>>>> said, I am not certain yet whether we would need such knowledge
>>>>>> because,
>>>>>> when using the Processor API, most of the work and decisions must be
>>>>>> done
>>>>>> by the user anyways.  It would matter though if the concept of
>>>>>> "triggers"
>>>>>> were to bubble up into the DSL because in the DSL the management of
>>>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>>>> Streams.
>>>>>>
>>>>>> [In any case, btw, we have the corner case where the user configured
>>>>>> the
>>>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>>>> extractor), at which point both punctuate variants are based on the
>>>>>> same
>>>>>> time semantics / timeline.]
>>>>>>
>>>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>>>> it
>>>>>> this far. :-)
>>>>>>
>>>>>> More than happy to hear your thoughts!
>>>>>> Michael
>>>>>>
>>>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>
>>>>>> Thanks Matthias.
>>>>>> Sure, will correct it right away.
>>>>>>
>>>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>> wrote:
>>>>>>
>>>>>> Thanks for preparing this page!
>>>>>>
>>>>>> About terminology:
>>>>>>
>>>>>> You introduce the term "event time" -- but we should call this
>>>>>> "stream
>>>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>>>> this
>>>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>>>
>>>>>> Does this make sense to you?
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>>
>>>>>> Thanks Ewen.
>>>>>>
>>>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>>>
>>>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>>
>>>>>>
>>>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>>>> public.
>>>>>>
>>>>>> --
>>>>>> Arun Mathew
>>>>>>
>>>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>>>>>> wrote:
>>>>>>
>>>>>>      Arun,
>>>>>>
>>>>>>      I've given you permission to edit the wiki. Let me know if
>>>>>> you run
>>>>>>
>>>>>> into any
>>>>>>
>>>>>>      issues.
>>>>>>
>>>>>>      -Ewen
>>>>>>
>>>>>>      On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp> <am...@yahoo-corp.jp>
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>>
>>>>>>      > Thanks Michal. I don\u2019t have the access yet [arunmathew88].
>>>>>> Should I
>>>>>>
>>>>>> be
>>>>>>
>>>>>>      > sending a separate mail for this?
>>>>>>      >
>>>>>>      > I thought one of the person following this thread would be
>>>>>> able to
>>>>>>
>>>>>> give me
>>>>>>
>>>>>>      > access.
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>>>>>      > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>      > *Date: *Friday, April 7, 2017 at 17:16
>>>>>>      > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>>      > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>>> semantics
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > Hi Arun,
>>>>>>      >
>>>>>>      > I was thinking along the same lines as you, listing the use
>>>>>> cases
>>>>>>
>>>>>> on the
>>>>>>
>>>>>>      > wiki, but didn't find time to get around doing that yet.
>>>>>>      > Don't mind if you do it if you have access now.
>>>>>>      > I was thinking it would be nice if, once we have the use
>>>>>> cases
>>>>>>
>>>>>> listed,
>>>>>>
>>>>>>      > people could use likes to up-vote the use cases similar to
>>>>>> what
>>>>>>
>>>>>> they're
>>>>>>
>>>>>>      > working on.
>>>>>>      >
>>>>>>      > I should have a bit more time to action this in the next
>>>>>> few days,
>>>>>>
>>>>>> but
>>>>>>
>>>>>>      > happy for you to do it if you can beat me to it ;-)
>>>>>>      >
>>>>>>      > Cheers,
>>>>>>      > Michal
>>>>>>      >
>>>>>>      > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>>      >
>>>>>>      > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > Of course. I was thinking of a subpage where people can
>>>>>>
>>>>>> collaborate.
>>>>>>
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > Will do as per Michael\u2019s suggestion.
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > Regards,
>>>>>>      >
>>>>>>      > Arun Mathew
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>>> <
>>>>>>
>>>>>> matthias@confluent.io> wrote:
>>>>>>
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >     Please share your Wiki-ID and a committer can give you
>>>>>> write
>>>>>>
>>>>>> access.
>>>>>>
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >     Btw: as you did not initiate the KIP, you should not
>>>>>> change the
>>>>>>
>>>>>> KIP
>>>>>>
>>>>>>      >
>>>>>>      >     without the permission of the original author -- in
>>>>>> this case
>>>>>>
>>>>>> Michael.
>>>>>>
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >     So you might also just share your thought over the
>>>>>> mailing list
>>>>>>
>>>>>> and
>>>>>>
>>>>>>      >
>>>>>>      >     Michael can update the KIP page. Or, as an alternative,
>>>>>> just
>>>>>>
>>>>>> create a
>>>>>>
>>>>>>      >
>>>>>>      >     subpage for the KIP page.
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >     @Michael: WDYT?
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >     -Matthias
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>>      >
>>>>>>      >     > Hi Jay,
>>>>>>      >
>>>>>>      >     >           Thanks for the advise, I would like to list
>>>>>> down
>>>>>>
>>>>>> the use cases as
>>>>>>
>>>>>>      >
>>>>>>      >     > per your suggestion. But it seems I don't have write
>>>>>>
>>>>>> permission to the
>>>>>>
>>>>>>      >
>>>>>>      >     > Apache Kafka Confluent Space. Whom shall I request
>>>>>> for it?
>>>>>>      >
>>>>>>      >     >
>>>>>>      >
>>>>>>      >     > Regarding your last question. We are using a patch in
>>>>>> our
>>>>>>
>>>>>> production system
>>>>>>
>>>>>>      >
>>>>>>      >     > which does exactly this.
>>>>>>      >
>>>>>>      >     > We window by the event time, but trigger punctuate in
>>>>>>
>>>>>> <punctuate interval>
>>>>>>
>>>>>>      >
>>>>>>      >     > duration of system time, in the absence of an event
>>>>>> crossing
>>>>>>
>>>>>> the punctuate
>>>>>>
>>>>>>      >
>>>>>>      >     > event time.
>>>>>>      >
>>>>>>      >     >
>>>>>>      >
>>>>>>      >     > We are using Kafka Stream for our Audit Trail, where
>>>>>> we need
>>>>>>
>>>>>> to output the
>>>>>>
>>>>>>      >
>>>>>>      >     > event counts on each topic on each cluster aggregated
>>>>>> over a
>>>>>>
>>>>>> 1 minute
>>>>>>
>>>>>>      >
>>>>>>      >     > window. We have to use event time to be able to cross
>>>>>> check
>>>>>>
>>>>>> the counts. But
>>>>>>
>>>>>>      >
>>>>>>      >     > we need to trigger punctuate [aggregate event pushes]
>>>>>> by
>>>>>>
>>>>>> system time in the
>>>>>>
>>>>>>      >
>>>>>>      >     > absence of events. Otherwise the event counts for
>>>>>> unexpired
>>>>>>
>>>>>> windows would
>>>>>>
>>>>>>      >
>>>>>>      >     > be 0 which is bad.
>>>>>>      >
>>>>>>      >     >
>>>>>>      >
>>>>>>      >     > "Maybe a hybrid solution works: I window by event
>>>>>> time but
>>>>>>
>>>>>> trigger results
>>>>>>
>>>>>>      >
>>>>>>      >     > by system time for windows that have updated? Not
>>>>>> really sure
>>>>>>
>>>>>> the details
>>>>>>
>>>>>>      >
>>>>>>      >     > of making that work. Does that work? Are there
>>>>>> concrete
>>>>>>
>>>>>> examples where you
>>>>>>
>>>>>>      >
>>>>>>      >     > actually want the current behavior?"
>>>>>>      >
>>>>>>      >     >
>>>>>>      >
>>>>>>      >     > --
>>>>>>      >
>>>>>>      >     > With Regards,
>>>>>>      >
>>>>>>      >     >
>>>>>>      >
>>>>>>      >     > Arun Mathew
>>>>>>      >
>>>>>>      >     > Yahoo! JAPAN Corporation
>>>>>>      >
>>>>>>      >     >
>>>>>>      >
>>>>>>      >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>>>
>>>>>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>>>>>
>>>>>>      >
>>>>>>      >     >
>>>>>>      >
>>>>>>      >     >> Hi Jay,
>>>>>>      >
>>>>>>      >     >>
>>>>>>      >
>>>>>>      >     >> The hybrid solution is exactly what I expect and
>>>>>> need for
>>>>>>
>>>>>> our use cases
>>>>>>
>>>>>>      >
>>>>>>      >     >> when dealing with telecom data.
>>>>>>      >
>>>>>>      >     >>
>>>>>>      >
>>>>>>      >     >> Thanks
>>>>>>      >
>>>>>>      >     >> Tianji
>>>>>>      >
>>>>>>      >     >>
>>>>>>      >
>>>>>>      >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>>>
>>>>>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>>>>>
>>>>>>      >
>>>>>>      >     >>
>>>>>>      >
>>>>>>      >     >>> Hey guys,
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>> One thing I've always found super important for
>>>>>> this kind
>>>>>>
>>>>>> of design work
>>>>>>
>>>>>>      >
>>>>>>      >     >> is
>>>>>>      >
>>>>>>      >     >>> to do a really good job of cataloging the landscape
>>>>>> of use
>>>>>>
>>>>>> cases and how
>>>>>>
>>>>>>      >
>>>>>>      >     >>> prevalent each one is. By that I mean not just
>>>>>> listing lots
>>>>>>
>>>>>> of uses, but
>>>>>>
>>>>>>      >
>>>>>>      >     >>> also grouping them into categories that
>>>>>> functionally need
>>>>>>
>>>>>> the same thing.
>>>>>>
>>>>>>      >
>>>>>>      >     >>> In the absence of this it is very hard to reason
>>>>>> about
>>>>>>
>>>>>> design proposals.
>>>>>>
>>>>>>      >
>>>>>>      >     >>> From the proposals so far I think we have a lot of
>>>>>>
>>>>>> discussion around
>>>>>>
>>>>>>      >
>>>>>>      >     >>> possible apis, but less around what the user needs
>>>>>> for
>>>>>>
>>>>>> different use
>>>>>>
>>>>>>      >
>>>>>>      >     >> cases
>>>>>>      >
>>>>>>      >     >>> and how they would implement that using the api.
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>> Here is an example:
>>>>>>      >
>>>>>>      >     >>> You aggregate click and impression data for a
>>>>>> reddit like
>>>>>>
>>>>>> site. Every ten
>>>>>>
>>>>>>      >
>>>>>>      >     >>> minutes you want to output a ranked list of the top
>>>>>> 10
>>>>>>
>>>>>> articles ranked by
>>>>>>
>>>>>>      >
>>>>>>      >     >>> clicks/impressions for each geographical area. I
>>>>>> want to be
>>>>>>
>>>>>> able run this
>>>>>>
>>>>>>      >
>>>>>>      >     >>> in steady state as well as rerun to regenerate
>>>>>> results (or
>>>>>>
>>>>>> catch up if it
>>>>>>
>>>>>>      >
>>>>>>      >     >>> crashes).
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>> There are a couple of tricky things that seem to
>>>>>> make this
>>>>>>
>>>>>> hard with
>>>>>>
>>>>>>      >
>>>>>>      >     >> either
>>>>>>      >
>>>>>>      >     >>> of the options proposed:
>>>>>>      >
>>>>>>      >     >>> 1. If I emit this data using event time I have the
>>>>>> problem
>>>>>>
>>>>>> described
>>>>>>
>>>>>>      >
>>>>>>      >     >> where
>>>>>>      >
>>>>>>      >     >>> a geographical region with no new clicks or
>>>>>> impressions
>>>>>>
>>>>>> will fail to
>>>>>>
>>>>>>      >
>>>>>>      >     >> output
>>>>>>      >
>>>>>>      >     >>> results.
>>>>>>      >
>>>>>>      >     >>> 2. If I emit this data using system time I have the
>>>>>> problem
>>>>>>
>>>>>> that when
>>>>>>
>>>>>>      >
>>>>>>      >     >>> reprocessing data my window may not be ten minutes
>>>>>> but 10
>>>>>>
>>>>>> hours if my
>>>>>>
>>>>>>      >
>>>>>>      >     >>> processing is very fast so it dramatically changes
>>>>>> the
>>>>>>
>>>>>> output.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>> Maybe a hybrid solution works: I window by event
>>>>>> time but
>>>>>>
>>>>>> trigger results
>>>>>>
>>>>>>      >
>>>>>>      >     >>> by system time for windows that have updated? Not
>>>>>> really
>>>>>>
>>>>>> sure the details
>>>>>>
>>>>>>      >
>>>>>>      >     >>> of making that work. Does that work? Are there
>>>>>> concrete
>>>>>>
>>>>>> examples where
>>>>>>
>>>>>>      >
>>>>>>      >     >> you
>>>>>>      >
>>>>>>      >     >>> actually want the current behavior?
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>> -Jay
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>>>
>>>>>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>>>>>
>>>>>>      >
>>>>>>      >     >>> wrote:
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>>> Hi All,
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> Thanks for the KIP. We were also in need of a
>>>>>> mechanism to
>>>>>>
>>>>>> trigger
>>>>>>
>>>>>>      >
>>>>>>      >     >>>> punctuate in the absence of events.
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> As I described in [
>>>>>>      >
>>>>>>      >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>>      >
>>>>>>      >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>>      >
>>>>>>      >     >>>> plugin.system.issuetabpanels:comment-
>>>>>> tabpanel#comment-
>>>>>>
>>>>>> 15926036
>>>>>>
>>>>>>      >
>>>>>>      >     >>>> ],
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>>    - Our approached involved using the event time
>>>>>> by
>>>>>>
>>>>>> default.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>    - The method to check if there is any punctuate
>>>>>> ready
>>>>>>
>>>>>> in the
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>    PunctuationQueue is triggered via the any event
>>>>>>
>>>>>> received by the
>>>>>>
>>>>>>      >
>>>>>>      >     >> stream
>>>>>>      >
>>>>>>      >     >>>>    tread, or at the polling intervals in the
>>>>>> absence of
>>>>>>
>>>>>> any events.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>    - When we create Punctuate objects (which
>>>>>> contains the
>>>>>>
>>>>>> next event
>>>>>>
>>>>>>      >
>>>>>>      >     >> time
>>>>>>      >
>>>>>>      >     >>>>    for punctuation and interval), we also record
>>>>>> the
>>>>>>
>>>>>> creation time
>>>>>>
>>>>>>      >
>>>>>>      >     >>> (system
>>>>>>      >
>>>>>>      >     >>>>    time).
>>>>>>      >
>>>>>>      >     >>>>    - While checking for maturity of Punctuate
>>>>>> Schedule by
>>>>>>      >
>>>>>>      >     >> mayBePunctuate
>>>>>>      >
>>>>>>      >     >>>>    method, we also check if the system clock has
>>>>>> elapsed
>>>>>>
>>>>>> the punctuate
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>    interval since the schedule creation time.
>>>>>>      >
>>>>>>      >     >>>>    - In the absence of any event, or in the
>>>>>> absence of any
>>>>>>
>>>>>> event for
>>>>>>
>>>>>>      >
>>>>>>      >     >> one
>>>>>>      >
>>>>>>      >     >>>>    topic in the partition group assigned to the
>>>>>> stream
>>>>>>
>>>>>> task, the system
>>>>>>
>>>>>>      >
>>>>>>      >     >>>> time
>>>>>>      >
>>>>>>      >     >>>>    will elapse the interval and we trigger a
>>>>>> punctuate
>>>>>>
>>>>>> using the
>>>>>>
>>>>>>      >
>>>>>>      >     >> expected
>>>>>>      >
>>>>>>      >     >>>>    punctuation event time.
>>>>>>      >
>>>>>>      >     >>>>    - we then create the next punctuation schedule
>>>>>> as
>>>>>>
>>>>>> punctuation event
>>>>>>
>>>>>>      >
>>>>>>      >     >>> time
>>>>>>      >
>>>>>>      >     >>>>    + punctuation interval, [again recording the
>>>>>> system
>>>>>>
>>>>>> time of creation
>>>>>>
>>>>>>      >
>>>>>>      >     >>> of
>>>>>>      >
>>>>>>      >     >>>> the
>>>>>>      >
>>>>>>      >     >>>>    schedule].
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>>> approach
>>>>>>
>>>>>> has pros and
>>>>>>
>>>>>>      >
>>>>>>      >     >>>> cons.
>>>>>>      >
>>>>>>      >     >>>> Pros
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>>    - Punctuates will happen in <punctuate
>>>>>> interval> time
>>>>>>
>>>>>> duration at
>>>>>>
>>>>>>      >
>>>>>>      >     >> max
>>>>>>      >
>>>>>>      >     >>> in
>>>>>>      >
>>>>>>      >     >>>>    terms of system time.
>>>>>>      >
>>>>>>      >     >>>>    - The semantics as a whole continues to revolve
>>>>>> around
>>>>>>
>>>>>> event time.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>    - We can use the old data [old timestamps] to
>>>>>> rerun any
>>>>>>
>>>>>> experiments
>>>>>>
>>>>>>      >
>>>>>>      >     >> or
>>>>>>      >
>>>>>>      >     >>>>    tests.
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> Cons
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>>    - In case the  <punctuate interval> is not a
>>>>>> time
>>>>>>
>>>>>> duration [say
>>>>>>
>>>>>>      >
>>>>>>      >     >>> logical
>>>>>>      >
>>>>>>      >     >>>>    time/event count], then the approach might not
>>>>>> be
>>>>>>
>>>>>> meaningful.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>    - In case there is a case where we have to wait
>>>>>> for an
>>>>>>
>>>>>> actual event
>>>>>>
>>>>>>      >
>>>>>>      >     >>> from
>>>>>>      >
>>>>>>      >     >>>>    a low event rate partition in the partition
>>>>>> group, this
>>>>>>
>>>>>> approach
>>>>>>
>>>>>>      >
>>>>>>      >     >> will
>>>>>>      >
>>>>>>      >     >>>> jump
>>>>>>      >
>>>>>>      >     >>>>    the gun.
>>>>>>      >
>>>>>>      >     >>>>    - in case the event processing cannot catch up
>>>>>> with the
>>>>>>
>>>>>> event rate
>>>>>>
>>>>>>      >
>>>>>>      >     >> and
>>>>>>      >
>>>>>>      >     >>>>    the expected timestamp events gets queued for
>>>>>> long
>>>>>>
>>>>>> time, this
>>>>>>
>>>>>>      >
>>>>>>      >     >> approach
>>>>>>      >
>>>>>>      >     >>>>    might jump the gun.
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> I believe the above approach and discussion goes
>>>>>> close to
>>>>>>
>>>>>> the approach
>>>>>>
>>>>>>      >
>>>>>>      >     >> A.
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> -----------
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> I like the idea of having an even count based
>>>>>> punctuate.
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> -----------
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> I agree with the discussion around approach C,
>>>>>> that we
>>>>>>
>>>>>> should provide
>>>>>>
>>>>>>      >
>>>>>>      >     >> the
>>>>>>      >
>>>>>>      >     >>>> user with the option to choose system time or
>>>>>> event time
>>>>>>
>>>>>> based
>>>>>>
>>>>>>      >
>>>>>>      >     >>> punctuates.
>>>>>>      >
>>>>>>      >     >>>> But I believe that the user predominantly wants to
>>>>>> use
>>>>>>
>>>>>> event time while
>>>>>>
>>>>>>      >
>>>>>>      >     >>> not
>>>>>>      >
>>>>>>      >     >>>> missing out on regular punctuates due to event
>>>>>> delays or
>>>>>>
>>>>>> event
>>>>>>
>>>>>>      >
>>>>>>      >     >> absences.
>>>>>>      >
>>>>>>      >     >>>> Hence a complex punctuate option as Matthias
>>>>>> mentioned
>>>>>>
>>>>>> (quoted below)
>>>>>>
>>>>>>      >
>>>>>>      >     >>> would
>>>>>>      >
>>>>>>      >     >>>> be most apt.
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> "- We might want to add "complex" schedules later
>>>>>> on
>>>>>>
>>>>>> (like, punctuate
>>>>>>
>>>>>>      >
>>>>>>      >     >> on
>>>>>>      >
>>>>>>      >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>>> time
>>>>>>
>>>>>> whatever comes
>>>>>>
>>>>>>      >
>>>>>>      >     >>>> first)."
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> -----------
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> I think I read somewhere that Kafka Streams
>>>>>> started with
>>>>>>
>>>>>> System Time as
>>>>>>
>>>>>>      >
>>>>>>      >     >>> the
>>>>>>      >
>>>>>>      >     >>>> punctuation standard, but was later changed to
>>>>>> Event Time.
>>>>>>
>>>>>> I guess
>>>>>>
>>>>>>      >
>>>>>>      >     >> there
>>>>>>      >
>>>>>>      >     >>>> would be some good reason behind it. As Kafka
>>>>>> Streams want
>>>>>>
>>>>>> to evolve
>>>>>>
>>>>>>      >
>>>>>>      >     >> more
>>>>>>      >
>>>>>>      >     >>>> on the Stream Processing front, I believe the
>>>>>> emphasis on
>>>>>>
>>>>>> event time
>>>>>>
>>>>>>      >
>>>>>>      >     >>> would
>>>>>>      >
>>>>>>      >     >>>> remain quite strong.
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> With Regards,
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> Arun Mathew
>>>>>>      >
>>>>>>      >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>>>
>>>>>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>>>>>
>>>>>>      >
>>>>>>      >     >> wrote:
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>>> Yeah I like PuncutationType much better; I just
>>>>>> threw
>>>>>>
>>>>>> Time out there
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>>> still
>>>>>>
>>>>>> think it's
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>> worth considering what this buys us over an
>>>>>> additional
>>>>>>
>>>>>> callback. I
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>> foresee a number of punctuate implementations
>>>>>> following
>>>>>>
>>>>>> this pattern:
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>> public void punctuate(PunctuationType type) {
>>>>>>      >
>>>>>>      >     >>>>>     switch (type) {
>>>>>>      >
>>>>>>      >     >>>>>         case EVENT_TIME:
>>>>>>      >
>>>>>>      >     >>>>>             methodA();
>>>>>>      >
>>>>>>      >     >>>>>             break;
>>>>>>      >
>>>>>>      >     >>>>>         case SYSTEM_TIME:
>>>>>>      >
>>>>>>      >     >>>>>             methodB();
>>>>>>      >
>>>>>>      >     >>>>>             break;
>>>>>>      >
>>>>>>      >     >>>>>     }
>>>>>>      >
>>>>>>      >     >>>>> }
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>> I guess one advantage of this approach is we
>>>>>> could add
>>>>>>
>>>>>> additional
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>> punctuation types later in a backwards compatible
>>>>>> way
>>>>>>
>>>>>> (like event
>>>>>>
>>>>>>      >
>>>>>>      >     >> count
>>>>>>      >
>>>>>>      >     >>>>> as you mentioned).
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>> -Tommy
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>>> Sax wrote:
>>>>>>      >
>>>>>>      >     >>>>>> That sounds promising.
>>>>>>      >
>>>>>>      >     >>>>>>
>>>>>>      >
>>>>>>      >     >>>>>> I am just wondering if `Time` is the best name.
>>>>>> Maybe we
>>>>>>
>>>>>> want to
>>>>>>
>>>>>>      >
>>>>>>      >     >> add
>>>>>>      >
>>>>>>      >     >>>>>> other non-time based punctuations at some point
>>>>>> later. I
>>>>>>
>>>>>> would
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>> suggest
>>>>>>      >
>>>>>>      >     >>>>>>
>>>>>>      >
>>>>>>      >     >>>>>> enum PunctuationType {
>>>>>>      >
>>>>>>      >     >>>>>>   EVENT_TIME,
>>>>>>      >
>>>>>>      >     >>>>>>   SYSTEM_TIME,
>>>>>>      >
>>>>>>      >     >>>>>> }
>>>>>>      >
>>>>>>      >     >>>>>>
>>>>>>      >
>>>>>>      >     >>>>>> or similar. Just to keep the door open -- it's
>>>>>> easier to
>>>>>>
>>>>>> add new
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>> stuff
>>>>>>      >
>>>>>>      >     >>>>>> if the name is more generic.
>>>>>>      >
>>>>>>      >     >>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>
>>>>>>      >
>>>>>>      >     >>>>>> -Matthias
>>>>>>      >
>>>>>>      >     >>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>
>>>>>>      >
>>>>>>      >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> I agree that the framework providing and
>>>>>> managing the
>>>>>>
>>>>>> notion of
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> stream
>>>>>>      >
>>>>>>      >     >>>>>>> time is valuable and not something we would
>>>>>> want to
>>>>>>
>>>>>> delegate to
>>>>>>
>>>>>>      >
>>>>>>      >     >> the
>>>>>>      >
>>>>>>      >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>>> separate
>>>>>>
>>>>>> callback
>>>>>>
>>>>>>      >
>>>>>>      >     >> (option
>>>>>>      >
>>>>>>      >     >>>>>>> C)
>>>>>>      >
>>>>>>      >     >>>>>>> is that messy (it could just be a default
>>>>>> method with
>>>>>>
>>>>>> an empty
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> implementation), but if we wanted a single API
>>>>>> to
>>>>>>
>>>>>> handle both
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> cases,
>>>>>>      >
>>>>>>      >     >>>>>>> how about something like the following?
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> enum Time {
>>>>>>      >
>>>>>>      >     >>>>>>>    STREAM,
>>>>>>      >
>>>>>>      >     >>>>>>>    CLOCK
>>>>>>      >
>>>>>>      >     >>>>>>> }
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> Then on ProcessorContext:
>>>>>>      >
>>>>>>      >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>>> We could
>>>>>>
>>>>>> allow
>>>>>>
>>>>>>      >
>>>>>>      >     >> this
>>>>>>      >
>>>>>>      >     >>>>>>> to
>>>>>>      >
>>>>>>      >     >>>>>>> be called once for each value of time to mix
>>>>>>
>>>>>> approaches.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> Then the Processor API becomes:
>>>>>>      >
>>>>>>      >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>>>
>>>>>> schedule resulted
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> in
>>>>>>      >
>>>>>>      >     >>>>>>> this call.
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> Thoughts?
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>>> Sax
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> I was thinking about the four options you
>>>>>> proposed in
>>>>>>
>>>>>> more
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> details
>>>>>>      >
>>>>>>      >     >>>>>>>> and
>>>>>>      >
>>>>>>      >     >>>>>>>> this are my thoughts:
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> (A) You argue, that users can still
>>>>>> "punctuate" on
>>>>>>
>>>>>> event-time
>>>>>>
>>>>>>      >
>>>>>>      >     >> via
>>>>>>      >
>>>>>>      >     >>>>>>>> process(), but I am not sure if this is
>>>>>> possible.
>>>>>>
>>>>>> Note, that
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> users
>>>>>>      >
>>>>>>      >     >>>>>>>> only
>>>>>>      >
>>>>>>      >     >>>>>>>> get record timestamps via context.timestamp().
>>>>>> Thus,
>>>>>>
>>>>>> users
>>>>>>
>>>>>>      >
>>>>>>      >     >> would
>>>>>>      >
>>>>>>      >     >>>>>>>> need
>>>>>>      >
>>>>>>      >     >>>>>>>> to
>>>>>>      >
>>>>>>      >     >>>>>>>> track the time progress per partition (based
>>>>>> on the
>>>>>>
>>>>>> partitions
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> they
>>>>>>      >
>>>>>>      >     >>>>>>>> obverse via context.partition(). (This alone
>>>>>> puts a
>>>>>>
>>>>>> huge burden
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> on
>>>>>>      >
>>>>>>      >     >>>>>>>> the
>>>>>>      >
>>>>>>      >     >>>>>>>> user by itself.) However, users are not
>>>>>> notified at
>>>>>>
>>>>>> startup
>>>>>>
>>>>>>      >
>>>>>>      >     >> what
>>>>>>      >
>>>>>>      >     >>>>>>>> partitions are assigned, and user are not
>>>>>> notified
>>>>>>
>>>>>> when
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> partitions
>>>>>>      >
>>>>>>      >     >>>>>>>> get
>>>>>>      >
>>>>>>      >     >>>>>>>> revoked. Because this information is not
>>>>>> available,
>>>>>>
>>>>>> it's not
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> possible
>>>>>>      >
>>>>>>      >     >>>>>>>> to
>>>>>>      >
>>>>>>      >     >>>>>>>> "manually advance" stream-time, and thus
>>>>>> event-time
>>>>>>
>>>>>> punctuation
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> within
>>>>>>      >
>>>>>>      >     >>>>>>>> process() seems not to be possible -- or do
>>>>>> you see a
>>>>>>
>>>>>> way to
>>>>>>
>>>>>>      >
>>>>>>      >     >> get
>>>>>>      >
>>>>>>      >     >>>>>>>> it
>>>>>>      >
>>>>>>      >     >>>>>>>> done? And even if, it might still be too
>>>>>> clumsy to
>>>>>>
>>>>>> use.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> (B) This does not allow to mix both
>>>>>> approaches, thus
>>>>>>
>>>>>> limiting
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> what
>>>>>>      >
>>>>>>      >     >>>>>>>> users
>>>>>>      >
>>>>>>      >     >>>>>>>> can do.
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> (C) This should give all flexibility we need.
>>>>>> However,
>>>>>>
>>>>>> just
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> adding
>>>>>>      >
>>>>>>      >     >>>>>>>> one
>>>>>>      >
>>>>>>      >     >>>>>>>> more method seems to be a solution that is too
>>>>>> simple
>>>>>>
>>>>>> (cf my
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> comments
>>>>>>      >
>>>>>>      >     >>>>>>>> below).
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>>> sure how
>>>>>>
>>>>>> a user
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> could
>>>>>>      >
>>>>>>      >     >>>>>>>> enable system-time and event-time punctuation
>>>>>> in
>>>>>>
>>>>>> parallel.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> Overall options (C) seems to be the most
>>>>>> promising
>>>>>>
>>>>>> approach to
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> me.
>>>>>>      >
>>>>>>      >     >>>>>>>> Because I also favor a clean API, we might
>>>>>> keep
>>>>>>
>>>>>> current
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> punctuate()
>>>>>>      >
>>>>>>      >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>>> at some
>>>>>>
>>>>>> later
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> point
>>>>>>      >
>>>>>>      >     >>>>>>>> when
>>>>>>      >
>>>>>>      >     >>>>>>>> people use the "new punctuate API".
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> Couple of follow up questions:
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> - I am wondering, if we should have two
>>>>>> callback
>>>>>>
>>>>>> methods or
>>>>>>
>>>>>>      >
>>>>>>      >     >> just
>>>>>>      >
>>>>>>      >     >>>>>>>> one
>>>>>>      >
>>>>>>      >     >>>>>>>> (ie, a unified for system and event time
>>>>>> punctuation
>>>>>>
>>>>>> or one for
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> each?).
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> - If we have one, how can the user figure out,
>>>>>> which
>>>>>>
>>>>>> condition
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> did
>>>>>>      >
>>>>>>      >     >>>>>>>> trigger?
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> - How would the API look like, for registering
>>>>>>
>>>>>> different
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> punctuate
>>>>>>      >
>>>>>>      >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> - We might want to add "complex" schedules
>>>>>> later on
>>>>>>
>>>>>> (like,
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> punctuate
>>>>>>      >
>>>>>>      >     >>>>>>>> on
>>>>>>      >
>>>>>>      >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>>> system-time
>>>>>>
>>>>>> whatever
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> comes
>>>>>>      >
>>>>>>      >     >>>>>>>> first). I don't say we should add this right
>>>>>> away, but
>>>>>>
>>>>>> we might
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> want
>>>>>>      >
>>>>>>      >     >>>>>>>> to
>>>>>>      >
>>>>>>      >     >>>>>>>> define the API in a way, that it allows
>>>>>> extensions
>>>>>>
>>>>>> like this
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> later
>>>>>>      >
>>>>>>      >     >>>>>>>> on,
>>>>>>      >
>>>>>>      >     >>>>>>>> without redesigning the API (ie, the API
>>>>>> should be
>>>>>>
>>>>>> designed
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> extensible)
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> - Did you ever consider count-based
>>>>>> punctuation?
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> I understand, that you would like to solve a
>>>>>> simple
>>>>>>
>>>>>> problem,
>>>>>>
>>>>>>      >
>>>>>>      >     >> but
>>>>>>      >
>>>>>>      >     >>>>>>>> we
>>>>>>      >
>>>>>>      >     >>>>>>>> learned from the past, that just "adding some
>>>>>> API"
>>>>>>
>>>>>> quickly
>>>>>>
>>>>>>      >
>>>>>>      >     >> leads
>>>>>>      >
>>>>>>      >     >>>>>>>> to a
>>>>>>      >
>>>>>>      >     >>>>>>>> not very well defined API that needs time
>>>>>> consuming
>>>>>>
>>>>>> clean up
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> later on
>>>>>>      >
>>>>>>      >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>>> holistic
>>>>>>      >
>>>>>>      >     >>>>>>>> punctuation
>>>>>>      >
>>>>>>      >     >>>>>>>> KIP
>>>>>>      >
>>>>>>      >     >>>>>>>> with this from the beginning on to avoid later
>>>>>> painful
>>>>>>      >
>>>>>>      >     >> redesign.
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> -Matthias
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> Thanks Thomas,
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> I'm also wary of changing the existing
>>>>>> semantics of
>>>>>>      >
>>>>>>      >     >> punctuate,
>>>>>>      >
>>>>>>      >     >>>>>>>>> for
>>>>>>      >
>>>>>>      >     >>>>>>>>> backward compatibility reasons, although I
>>>>>> like the
>>>>>>      >
>>>>>>      >     >> conceptual
>>>>>>      >
>>>>>>      >     >>>>>>>>> simplicity of that option.
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>>> a way,
>>>>>>
>>>>>> uglier.
>>>>>>
>>>>>>      >
>>>>>>      >     >> I
>>>>>>      >
>>>>>>      >     >>>>>>>>> added
>>>>>>      >
>>>>>>      >     >>>>>>>>> this to the KIP now as option (C).
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>>> more
>>>>>>
>>>>>> flexible,
>>>>>>
>>>>>>      >
>>>>>>      >     >> as
>>>>>>      >
>>>>>>      >     >>>>>>>>> it
>>>>>>      >
>>>>>>      >     >>>>>>>>> allows
>>>>>>      >
>>>>>>      >     >>>>>>>>> you to return any value, you're not limited
>>>>>> to event
>>>>>>
>>>>>> time or
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> system
>>>>>>      >
>>>>>>      >     >>>>>>>>> time
>>>>>>      >
>>>>>>      >     >>>>>>>>> (although I don't see an actual use case
>>>>>> where you
>>>>>>
>>>>>> might need
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> anything
>>>>>>      >
>>>>>>      >     >>>>>>>>> else then those two). Hence I also proposed
>>>>>> the
>>>>>>
>>>>>> option to
>>>>>>
>>>>>>      >
>>>>>>      >     >> allow
>>>>>>      >
>>>>>>      >     >>>>>>>>> users
>>>>>>      >
>>>>>>      >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>>> for
>>>>>>
>>>>>> them given
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> the
>>>>>>      >
>>>>>>      >     >>>>>>>>> presence or absence of messages, much like
>>>>>> they can
>>>>>>
>>>>>> decide
>>>>>>
>>>>>>      >
>>>>>>      >     >> what
>>>>>>      >
>>>>>>      >     >>>>>>>>> msg
>>>>>>      >
>>>>>>      >     >>>>>>>>> time
>>>>>>      >
>>>>>>      >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>>> What do
>>>>>>
>>>>>> you
>>>>>>
>>>>>>      >
>>>>>>      >     >> think
>>>>>>      >
>>>>>>      >     >>>>>>>>> about
>>>>>>      >
>>>>>>      >     >>>>>>>>> that? This is probably most flexible but also
>>>>>> most
>>>>>>      >
>>>>>>      >     >> complicated.
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> All comments appreciated.
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> Cheers,
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> Michal
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>>      >
>>>>>>      >     >>>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>> Although I fully agree we need a way to
>>>>>> trigger
>>>>>>
>>>>>> periodic
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>> processing
>>>>>>      >
>>>>>>      >     >>>>>>>>>> that is independent from whether and when
>>>>>> messages
>>>>>>
>>>>>> arrive,
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>> I'm
>>>>>>      >
>>>>>>      >     >>>>>>>>>> not sure
>>>>>>      >
>>>>>>      >     >>>>>>>>>> I like the idea of changing the existing
>>>>>> semantics
>>>>>>
>>>>>> across
>>>>>>
>>>>>>      >
>>>>>>      >     >> the
>>>>>>      >
>>>>>>      >     >>>>>>>>>> board.
>>>>>>      >
>>>>>>      >     >>>>>>>>>> What if we added an additional callback to
>>>>>> Processor
>>>>>>
>>>>>> that
>>>>>>
>>>>>>      >
>>>>>>      >     >> can
>>>>>>      >
>>>>>>      >     >>>>>>>>>> be
>>>>>>      >
>>>>>>      >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>>> always
>>>>>>
>>>>>> called at
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>> fixed, wall
>>>>>>      >
>>>>>>      >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>>> have to
>>>>>>
>>>>>> give
>>>>>>
>>>>>>      >
>>>>>>      >     >> up
>>>>>>      >
>>>>>>      >     >>>>>>>>>> the
>>>>>>      >
>>>>>>      >     >>>>>>>>>> notion
>>>>>>      >
>>>>>>      >     >>>>>>>>>> of stream time to be able to do periodic
>>>>>> processing.
>>>>>>      >
>>>>>>      >     >>>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>>> Borowiecki
>>>>>>
>>>>>> wrote:
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> Hi all,
>>>>>>      >
>>>>>>      >     >>>>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>>>
>>>>>> punctuate
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> semantics
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>>>
>>>>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/ confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>>
>>>>>>      >
>>>>>>      >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>>>
>>>>>> 138%
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> 3A+C
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> hange+
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> punctuate+semantics>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> .
>>>>>>      >
>>>>>>      >     >>>>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> Appreciating there can be different views
>>>>>> on
>>>>>>
>>>>>> system-time
>>>>>>
>>>>>>      >
>>>>>>      >     >> vs
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> event-
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> time
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>>> case and
>>>>>>
>>>>>> the
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> importance of
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>>> I've
>>>>>>
>>>>>> left it
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> quite
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> open
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> and
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>>>
>>>>>> progresses.
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> Thanks,
>>>>>>      >
>>>>>>      >     >>>>>>>>>>> Michal
>>>>>>      >
>>>>>>      >     >>>>>>> --
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>     Tommy Becker
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>     Senior Software Engineer
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>     tivo.com
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> ________________________________
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> This email and any attachments may contain
>>>>>> confidential
>>>>>>
>>>>>> and
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> privileged material for the sole use of the
>>>>>> intended
>>>>>>
>>>>>> recipient.
>>>>>>
>>>>>>      >
>>>>>>      >     >> Any
>>>>>>      >
>>>>>>      >     >>>>>>> review, copying, or distribution of this email
>>>>>> (or any
>>>>>>      >
>>>>>>      >     >> attachments)
>>>>>>      >
>>>>>>      >     >>>>>>> by others is prohibited. If you are not the
>>>>>> intended
>>>>>>
>>>>>> recipient,
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> please contact the sender immediately and
>>>>>> permanently
>>>>>>
>>>>>> delete this
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> email and any attachments. No employee or agent
>>>>>> of TiVo
>>>>>>
>>>>>> Inc. is
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> authorized to conclude any binding agreement on
>>>>>> behalf
>>>>>>
>>>>>> of TiVo
>>>>>>
>>>>>>      >
>>>>>>      >     >> Inc.
>>>>>>      >
>>>>>>      >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>>> only be
>>>>>>
>>>>>> made by a
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>>>> signed written agreement.
>>>>>>      >
>>>>>>      >     >>>>>>>
>>>>>>      >
>>>>>>      >     >>>>> --
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>>     Tommy Becker
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>>     Senior Software Engineer
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>>     tivo.com
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>> ________________________________
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>> This email and any attachments may contain
>>>>>> confidential
>>>>>>
>>>>>> and
>>>>>>
>>>>>>      >
>>>>>>      >     >> privileged
>>>>>>      >
>>>>>>      >     >>>>> material for the sole use of the intended
>>>>>> recipient. Any
>>>>>>
>>>>>> review,
>>>>>>
>>>>>>      >
>>>>>>      >     >>> copying,
>>>>>>      >
>>>>>>      >     >>>>> or distribution of this email (or any
>>>>>> attachments) by
>>>>>>
>>>>>> others is
>>>>>>
>>>>>>      >
>>>>>>      >     >>>> prohibited.
>>>>>>      >
>>>>>>      >     >>>>> If you are not the intended recipient, please
>>>>>> contact the
>>>>>>
>>>>>> sender
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>> immediately and permanently delete this email and
>>>>>> any
>>>>>>
>>>>>> attachments. No
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>>> conclude
>>>>>>
>>>>>> any binding
>>>>>>
>>>>>>      >
>>>>>>      >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>>> Binding
>>>>>>
>>>>>> agreements with
>>>>>>
>>>>>>      >
>>>>>>      >     >> TiVo
>>>>>>      >
>>>>>>      >     >>>>> Inc. may only be made by a signed written
>>>>>> agreement.
>>>>>>      >
>>>>>>      >     >>>>>
>>>>>>      >
>>>>>>      >     >>>>
>>>>>>      >
>>>>>>      >     >>>
>>>>>>      >
>>>>>>      >     >>
>>>>>>      >
>>>>>>      >     >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > --
>>>>>>      >
>>>>>>      > <http://www.openbet.com/> <http://www.openbet.com/>
>>>>>>
>>>>>>      >
>>>>>>      > *Michal Borowiecki*
>>>>>>      >
>>>>>>      > *Senior Software Engineer L4*
>>>>>>      >
>>>>>>      > *T: *
>>>>>>      >
>>>>>>      > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>>>      >
>>>>>>      > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > *E: *
>>>>>>      >
>>>>>>      > michal.borowiecki@openbet.com
>>>>>>      >
>>>>>>      > *W: *
>>>>>>      >
>>>>>>      > www.openbet.com
>>>>>>      >
>>>>>>      > *OpenBet Ltd*
>>>>>>      >
>>>>>>      > Chiswick Park Building 9
>>>>>>      >
>>>>>>      > 566 Chiswick High Rd
>>>>>>      >
>>>>>>      > London
>>>>>>      >
>>>>>>      > W4 5XT
>>>>>>      >
>>>>>>      > UK
>>>>>>      >
>>>>>>      > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>      > This message is confidential and intended only for the
>>>>>> addressee.
>>>>>>
>>>>>> If you
>>>>>>
>>>>>>      > have received this message in error, please immediately
>>>>>> notify the
>>>>>>      > postmaster@openbet.com and delete it from your system as
>>>>>> well as
>>>>>>
>>>>>> any
>>>>>>
>>>>>>      > copies. The content of e-mails as well as traffic data may
>>>>>> be
>>>>>>
>>>>>> monitored by
>>>>>>
>>>>>>      > OpenBet for employment and security purposes. To protect
>>>>>> the
>>>>>>
>>>>>> environment
>>>>>>
>>>>>>      > please do not print this e-mail unless necessary. OpenBet
>>>>>> Ltd.
>>>>>>
>>>>>> Registered
>>>>>>
>>>>>>      > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>>> London,
>>>>>>
>>>>>> W4 5XT,
>>>>>>
>>>>>>      > United Kingdom. A company registered in England and Wales.
>>>>>>
>>>>>> Registered no.
>>>>>>
>>>>>>      > 3134634. VAT no. GB927523612
>>>>>>      >
>>>>>>      >
>>>>>>      >
>>>>>>
>>>>>>
>>>>>>
>>>>>> --
>>>>>>
>>>>>>
>>>>>>      Tommy Becker
>>>>>>
>>>>>>      Senior Software Engineer
>>>>>>
>>>>>>      O +1 919.460.4747 <%28919%29%20460-4747>
>>>>>>
>>>>>>
>>>>>>      tivo.com
>>>>>>
>>>>>>
>>>>>>
>>>>>> ________________________________
>>>>>>
>>>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>>>
>>>>>>
>>>>>> --
>>>>>> <http://www.openbet.com/> Michal Borowiecki
>>>>>> Senior Software Engineer L4
>>>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>>>
>>>>>>
>>>>>>
>>> -- 
>>> Signature
>>> <http://www.openbet.com/> 	Michal Borowiecki
>>> Senior Software Engineer L4
>>> 	T: 	+44 208 742 1600
>>>
>>> 	
>>> 	+44 203 249 8448
>>>
>>> 	
>>> 	
>>> 	E: 	michal.borowiecki@openbet.com
>>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>>
>>> 	
>>> 	OpenBet Ltd
>>>
>>> 	Chiswick Park Building 9
>>>
>>> 	566 Chiswick High Rd
>>>
>>> 	London
>>>
>>> 	W4 5XT
>>>
>>> 	UK
>>>
>>> 	
>>> <https://www.openbet.com/email_promo>
>>>
>>> This message is confidential and intended only for the addressee. If
>>> you have received this message in error, please immediately notify the
>>> postmaster@openbet.com <ma...@openbet.com> and delete it
>>> from your system as well as any copies. The content of e-mails as well
>>> as traffic data may be monitored by OpenBet for employment and
>>> security purposes. To protect the environment please do not print this
>>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>>> company registered in England and Wales. Registered no. 3134634. VAT
>>> no. GB927523612
>>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If you
>> have received this message in error, please immediately notify the
>> postmaster@openbet.com <ma...@openbet.com> and delete it
>> from your system as well as any copies. The content of e-mails as well
>> as traffic data may be monitored by OpenBet for employment and security
>> purposes. To protect the environment please do not print this e-mail
>> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
>> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>> registered in England and Wales. Registered no. 3134634. VAT no.
>> GB927523612
>>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Hi,

I do like Damian's API proposal about the punctuation callback function.

I also did reread the KIP and thought about the semantics we want to
provide.

> Given the above, I don't see a reason any more for a separate system-time based punctuation.

I disagree here. There are real-time applications, that want to get
callbacks in regular system-time intervals (completely independent from
stream-time). Thus we should allow this -- if we really follow the
"hybrid" approach, this could be configured with stream-time interval
infinite and delay whatever system-time punctuation interval you want to
have. However, I would like to add a proper API for this and do this
configuration under the hood (that would allow one implementation within
all kind of branching for different cases).

Thus, we definitely should have PunctutionType#StreamTime and
#SystemTime -- and additionally, we _could_ have #Hybrid. Thus, I am not
a fan of your latest API proposal.


About the hybrid approach in general. On the one hand I like it, on the
other hand, it seems to be rather (1) complicated (not necessarily from
an implementation point of view, but for people to understand it) and
(2) mixes two semantics together in a "weird" way". Thus, I disagree with:

> It may appear complicated at first but I do think these semantics will
> still be more understandable to users than having 2 separate punctuation
> schedules/callbacks with different PunctuationTypes.

This statement only holds if you apply strong assumptions that I don't
believe hold in general -- see (2) for details -- and I think it is
harder than you assume to reason about the hybrid approach in general.
IMHO, the hybrid approach is a "false friend" that seems to be easy to
reason about...


(1) Streams always embraced "easy to use" and we should really be
careful to keep it this way. On the other hand, as we are talking about
changes to PAPI, it won't affect DSL users (DSL does not use punctuation
at all at the moment), and thus, the "easy to use" mantra might not be
affected, while it will allow advanced users to express more complex stuff.

I like the mantra: "make simple thing easy and complex things possible".

(2) IMHO the major disadvantage (issue?) of the hybrid approach is the
implicit assumption that even-time progresses at the same "speed" as
system-time during regular processing. This implies the assumption that
a slower progress in stream-time indicates the absence of input events
(and that later arriving input events will have a larger event-time with
high probability). Even if this might be true for some use cases, I
doubt it holds in general. Assume that you get a spike in traffic and
for some reason stream-time does advance slowly because you have more
records to process. This might trigger a system-time based punctuation
call even if this seems not to be intended. I strongly believe that it
is not easy to reason about the semantics of the hybrid approach (even
if the intentional semantics would be super useful -- but I doubt that
we get want we ask for).

Thus, I also believe that one might need different "configuration"
values for the hybrid approach if you run the same code for different
scenarios: regular processing, re-processing, catching up scenario. And
as the term "configuration" implies, we might be better off to not mix
configuration with business logic that is expressed via code.


One more comment: I also don't think that the hybrid approach is
deterministic as claimed in the use-case subpage. I understand the
reasoning and agree, that it is deterministic if certain assumptions
hold -- compare above -- and if configured correctly. But strictly
speaking it's not because there is a dependency on system-time (and
IMHO, if system-time is involved it cannot be deterministic by definition).


> I see how in theory this could be implemented on top of the 2 punctuate
> callbacks with the 2 different PunctuationTypes (one stream-time based,
> the other system-time based) but it would be a much more complicated
> scheme and I don't want to suggest that.

I agree that expressing the intended hybrid semantics is harder if we
offer only #StreamTime and #SystemTime punctuation. However, I also
believe that the hybrid approach is a "false friend" with regard to
reasoning about the semantics (it indicates that it more easy as it is
in reality). Therefore, we might be better off to not offer the hybrid
approach and make it clear to a developed, that it is hard to mix
#StreamTime and #SystemTime in a semantically sound way.


Looking forward to your feedback. :)

-Matthias




On 4/22/17 11:43 AM, Michal Borowiecki wrote:
> Hi all,
> 
> Looking for feedback on the functional interface approach Damian
> proposed. What do people think?
> 
> Further on the semantics of triggering punctuate though:
> 
> I ran through the 2 use cases that Arun had kindly put on the wiki
> (https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases)
> in my head and on a whiteboard and I can't find a better solution than
> the "hybrid" approach he had proposed.
> 
> I see how in theory this could be implemented on top of the 2 punctuate
> callbacks with the 2 different PunctuationTypes (one stream-time based,
> the other system-time based) but it would be a much more complicated
> scheme and I don't want to suggest that.
> 
> However, to add to the hybrid algorithm proposed, I suggest one
> parameter to that: a tolerance period, expressed in milliseconds
> system-time, after which the punctuation will be invoked in case the
> stream-time advance hasn't triggered it within the requested interval
> since the last invocation of punctuate (as recorded in system-time)
> 
> This would allow a user-defined tolerance for late arriving events. The
> trade off would be left for the user to decide: regular punctuation in
> the case of absence of events vs allowing for records arriving late or
> some build-up due to processing not catching up with the event rate.
> In the one extreme, this tolerance could be set to infinity, turning
> hybrid into simply stream-time based punctuate, like we have now. In the
> other extreme, the tolerance could be set to 0, resulting in a
> system-time upper bound on the effective punctuation interval.
> 
> Given the above, I don't see a reason any more for a separate
> system-time based punctuation. The "hybrid" approach with 0ms tolerance
> would under normal operation trigger at regular intervals wrt the
> system-time, except in cases of re-play/catch-up, where the stream time
> advances faster than system time. In these cases punctuate would happen
> more often than the specified interval wrt system time. However, the
> use-cases that need system-time punctuations (that I've seen at least)
> really only have a need for an upper bound on punctuation delay but
> don't need a lower bound.
> 
> To that effect I'd propose the api to be as follows, on ProcessorContext:
> 
> schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)
> 
> schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate
> 
> Punctuation is triggered when either:
> - the stream time advances past the (stream time of the previous
> punctuation) + interval;
> - or (iff the toleranceInterval is set) when the system time advances
> past the (system time of the previous punctuation) + interval +
> toleranceInterval
> 
> In either case:
> - we trigger punctuate passing as the argument the stream time at which
> the current punctuation was meant to happen
> - next punctuate is scheduled at (stream time at which the current
> punctuation was meant to happen) + interval
> 
> It may appear complicated at first but I do think these semantics will
> still be more understandable to users than having 2 separate punctuation
> schedules/callbacks with different PunctuationTypes.
> 
> 
> 
> PS. Having re-read this, maybe the following alternative would be easier
> to understand (WDYT?):
> 
> schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval
> 
> schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate
> 
> Punctuation is triggered when either:
> - the stream time advances past the (stream time of the previous
> punctuation) + streamTimeInterval;
> - or (iff systemTimeUpperBound is set) when the system time advances
> past the (system time of the previous punctuation) + systemTimeUpperBound
> 
> Awaiting comments.
> 
> Thanks,
> Michal
> 
> On 21/04/17 16:56, Michal Borowiecki wrote:
>>
>> Yes, that's what I meant. Just wanted to highlight we'd deprecate it
>> in favour of something that doesn't return a record. Not a problem though.
>>
>>
>> On 21/04/17 16:32, Damian Guy wrote:
>>> Thanks Michal,
>>> I agree Transformer.punctuate should also be void, but we can deprecate
>>> that too in favor of the new interface.
>>>
>>> Thanks for the javadoc PR!
>>>
>>> Cheers,
>>> Damian
>>>
>>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>>> michal.borowiecki@openbet.com> wrote:
>>>
>>>> Yes, that looks better to me.
>>>>
>>>> Note that punctuate on Transformer is currently returning a record, but I
>>>> think it's ok to have all output records be sent via
>>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>>> multiple records from one invocation of punctuate.
>>>>
>>>> This way it's consistent between Processor and Transformer.
>>>>
>>>>
>>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>>> there:
>>>>
>>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>>
>>>> and PR: https://github.com/apache/kafka/pull/2884
>>>>
>>>> Cheers,
>>>>
>>>> Michal
>>>> On 20/04/17 18:55, Damian Guy wrote:
>>>>
>>>> Hi Michal,
>>>>
>>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>>> the API.
>>>> 1. deprecate the punctuate method on Processor
>>>> 2. create a new Functional Interface just for Punctuation, something like:
>>>> interface Punctuator {
>>>>     void punctuate(long timestamp)
>>>> }
>>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>>> interval, PunctuationType type, Punctuator callback)
>>>> 4. deprecate the existing schedule function
>>>>
>>>> Thoughts?
>>>>
>>>> Thanks,
>>>> Damian
>>>>
>>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>>> michal.borowiecki@openbet.com> wrote:
>>>>
>>>>> Hi Thomas,
>>>>>
>>>>> I would say our use cases fall in the same category as yours.
>>>>>
>>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>>
>>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>>> of design:
>>>>>
>>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>>
>>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>>
>>>>> Since the output field X is derived in some non-trivial way, we don't
>>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>>> another input field Z).
>>>>>
>>>>> So we have kv stores with the records and an additional kv store with
>>>>> timestamp->id mapping which act like an index where we periodically do a
>>>>> ranged query.
>>>>>
>>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>>> work when there were no regular msgs on the input topic.
>>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>>> outside we created a "ticker" that produced msgs once per second onto
>>>>> another topic and fed it into the same topology to trigger punctuate.
>>>>> This didn't work either, which was much more surprising to us at the
>>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>>> *all* input partitions receive messages regularly.
>>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>>> is consumed by the second topology and is its only input topic. There's a
>>>>> transformer on that topic which populates and updates the time-based
>>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>>> elapsed, the record id is sent to the main transformer, which
>>>>> updates/deletes the record from the main kv store and forwards the
>>>>> transformed record to the output topic.
>>>>>
>>>>> To me this setup feels horrendously complicated for what it does.
>>>>>
>>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>>> However, the ticks don't feel so hacky when you realise they give you
>>>>> some hypothetical benefits in predictability. You can reprocess the
>>>>> messages in a reproducible manner, since the topologies use event-time,
>>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>>> reprocessing).
>>>>> To make that work though, we would have to have the stream time advance
>>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>>> presence of messages on the other input topic.
>>>>>
>>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>>> and the hybrid would work to simplify this a lot.
>>>>>
>>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>>> have to go with punctuate there too.
>>>>>
>>>>> Thanks,
>>>>> Michal
>>>>>
>>>>>
>>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>>
>>>>> Here's an example that we currently have.  We have a streams processor
>>>>> that does a transform from one topic into another. One of the fields in
>>>>> the source topic record is an expiration time, and one of the functions
>>>>> of the processor is to ensure that expired records get deleted promptly
>>>>> after that time passes (typically days or weeks after the message was
>>>>> originally produced). To do that, the processor keeps a state store of
>>>>> keys and expiration times, iterates that store in punctuate(), and
>>>>> emits delete (null) records for expired items. This needs to happen at
>>>>> some minimum interval regardless of the incoming message rate of the
>>>>> source topic.
>>>>>
>>>>> In this scenario, the expiration of records is the primary function of
>>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>>> range of use-cases.
>>>>>
>>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>>
>>>>> I apologize for the longer email below.  To my defense, it started
>>>>> out much
>>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>>> devil's
>>>>> advocate for a number of arguments brought forth in order to help
>>>>> improve
>>>>> this KIP -- I am not implying I necessarily disagree with the
>>>>> arguments.
>>>>>
>>>>> That aside, here are some further thoughts.
>>>>>
>>>>> First, there are (at least?) two categories for actions/behavior you
>>>>> invoke
>>>>> via punctuate():
>>>>>
>>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>>> to
>>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>>> the
>>>>> impact of punctuate is typically not observable by other processing
>>>>> nodes
>>>>> in the topology.
>>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>>> the
>>>>> punctuate is all about being observable by downstream processing
>>>>> nodes.
>>>>>
>>>>> A few releases back, we introduced record caches (DSL) and state
>>>>> store
>>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>>> relating to
>>>>> (2) where some users needed to control -- here: limit -- the
>>>>> downstream
>>>>> output rate of Kafka Streams because the downstream systems/apps
>>>>> would not
>>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>>> their
>>>>> scalability).  The argument for KIP-63, which notably did not
>>>>> introduce a
>>>>> "trigger" API, was that such an interaction with downstream systems
>>>>> is an
>>>>> operational concern;  it should not impact the processing *logic* of
>>>>> your
>>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>>> API,
>>>>> especially not the declarative DSL, with such operational concerns.
>>>>>
>>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>>> sorry, I
>>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>>> observing that our conversation is moving more and more into the
>>>>> direction
>>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>>> for
>>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>>> comments voiced here are about sth like "IF stream-time didn't
>>>>> trigger
>>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>>> want
>>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>>> related
>>>>> note, whatever we are discussing here will impact state store caches
>>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>>> should
>>>>> clarify any such impact here.
>>>>>
>>>>> Switching topics slightly.
>>>>>
>>>>> Jay wrote:
>>>>>
>>>>> One thing I've always found super important for this kind of design
>>>>> work
>>>>> is to do a really good job of cataloging the landscape of use cases
>>>>> and
>>>>> how prevalent each one is.
>>>>>
>>>>> +1 to this, as others have already said.
>>>>>
>>>>> Here, let me highlight -- just in case -- that when we talked about
>>>>> windowing use cases in the recent emails, the Processor API (where
>>>>> `punctuate` resides) does not have any notion of windowing at
>>>>> all.  If you
>>>>> want to do windowing *in the Processor API*, you must do so manually
>>>>> in
>>>>> combination with window stores.  For this reason I'd suggest to
>>>>> discuss use
>>>>> cases not just in general, but also in view of how you'd do so in the
>>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>>> `punctuate`
>>>>> does not impact the DSL at all, unless we add new functionality to
>>>>> it.
>>>>>
>>>>> Jay wrote in his strawman example:
>>>>>
>>>>> You aggregate click and impression data for a reddit like site.
>>>>> Every ten
>>>>> minutes you want to output a ranked list of the top 10 articles
>>>>> ranked by
>>>>> clicks/impressions for each geographical area. I want to be able
>>>>> run this
>>>>> in steady state as well as rerun to regenerate results (or catch up
>>>>> if it
>>>>> crashes).
>>>>>
>>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>>> we
>>>>> argued that the reason for saying "every ten minutes" above is not
>>>>> necessarily about because you want to output data *exactly* after ten
>>>>> minutes, but that you want to perform an aggregation based on 10-
>>>>> minute
>>>>> windows of input data; i.e., the point is about specifying the input
>>>>> for
>>>>> your aggregation, not or less about when the results of the
>>>>> aggregation
>>>>> should be send downstream.  To take an extreme example, you could
>>>>> disable
>>>>> record caches and let your app output a downstream update for every
>>>>> incoming input record.  If the last input record was from at minute 7
>>>>> of 10
>>>>> (for a 10-min window), then what your app would output at minute 10
>>>>> would
>>>>> be identical to what it had already emitted at minute 7 earlier
>>>>> anyways.
>>>>> This is particularly true when we take late-arriving data into
>>>>> account:  if
>>>>> a late record arrived at minute 13, your app would (by default) send
>>>>> a new
>>>>> update downstream, even though the "original" 10 minutes have already
>>>>> passed.
>>>>>
>>>>> Jay wrote...:
>>>>>
>>>>> There are a couple of tricky things that seem to make this hard
>>>>> with
>>>>>
>>>>> either
>>>>>
>>>>> of the options proposed:
>>>>> 1. If I emit this data using event time I have the problem
>>>>> described where
>>>>> a geographical region with no new clicks or impressions will fail
>>>>> to
>>>>>
>>>>> output
>>>>>
>>>>> results.
>>>>>
>>>>> ...and Arun Mathew wrote:
>>>>>
>>>>>
>>>>> We window by the event time, but trigger punctuate in <punctuate
>>>>> interval>
>>>>> duration of system time, in the absence of an event crossing the
>>>>> punctuate
>>>>> event time.
>>>>>
>>>>> So, given what I wrote above about the status quo and what you can
>>>>> already
>>>>> do with it, is the concern that the state store cache doesn't give
>>>>> you
>>>>> *direct* control over "forcing an output after no later than X
>>>>> seconds [of
>>>>> processing-time]" but only indirect control through a cache
>>>>> size?  (Note
>>>>> that I am not dismissing the claims why this might be helpful.)
>>>>>
>>>>> Arun Mathew wrote:
>>>>>
>>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>>> output the
>>>>> event counts on each topic on each cluster aggregated over a 1
>>>>> minute
>>>>> window. We have to use event time to be able to cross check the
>>>>> counts.
>>>>>
>>>>> But
>>>>>
>>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>>> time in
>>>>>
>>>>> the
>>>>>
>>>>> absence of events. Otherwise the event counts for unexpired windows
>>>>> would
>>>>> be 0 which is bad.
>>>>>
>>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>>> absence
>>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>>> Scala
>>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>>> root
>>>>> cause that the downstream system interprets the absence of output in
>>>>> a
>>>>> particular way ("No output after 1 minute = I consider the output to
>>>>> be
>>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>>> possible)
>>>>> to correctly handle the difference between absence of output vs.
>>>>> output of
>>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>>> but
>>>>> want to understand the motivation better. :-)
>>>>>
>>>>> Also, to add some perspective, in some related discussions we talked
>>>>> about
>>>>> how a Kafka Streams application should not worry or not be coupled
>>>>> unnecessarily with such interpretation specifics in a downstream
>>>>> system's
>>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>>> more
>>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>>> than
>>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>>> and
>>>>> downstream systems, including helping to reconcile the differences in
>>>>> how
>>>>> these systems interpret changes, updates, late-arriving data,
>>>>> etc.  Kafka
>>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>>> processing from the specifics of downstream systems, thanks to
>>>>> specific
>>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>>> this
>>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>>> but
>>>>> it's currently awkward to use Connect for this", this might be a
>>>>> problem we
>>>>> can solve, too.)
>>>>>
>>>>> Switching topics slightly again.
>>>>>
>>>>> Thomas wrote:
>>>>>
>>>>> I'm not entirely convinced that a separate callback (option C)
>>>>> is that messy (it could just be a default method with an empty
>>>>> implementation), but if we wanted a single API to handle both
>>>>> cases,
>>>>> how about something like the following?
>>>>>
>>>>> enum Time {
>>>>>    STREAM,
>>>>>    CLOCK
>>>>> }
>>>>>
>>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>>> then
>>>>> whatever the user is doing inside this method is a black box to Kafka
>>>>> Streams (similar to how we have no idea what the user does inside a
>>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>>> won't
>>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>>> frequency than the processing-time action.  Or, we won't know whether
>>>>> the
>>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>>> minute of
>>>>> stream-time or 1-minute of processing-time, whichever comes
>>>>> first").  That
>>>>> said, I am not certain yet whether we would need such knowledge
>>>>> because,
>>>>> when using the Processor API, most of the work and decisions must be
>>>>> done
>>>>> by the user anyways.  It would matter though if the concept of
>>>>> "triggers"
>>>>> were to bubble up into the DSL because in the DSL the management of
>>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>>> Streams.
>>>>>
>>>>> [In any case, btw, we have the corner case where the user configured
>>>>> the
>>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>>> extractor), at which point both punctuate variants are based on the
>>>>> same
>>>>> time semantics / timeline.]
>>>>>
>>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>>> it
>>>>> this far. :-)
>>>>>
>>>>> More than happy to hear your thoughts!
>>>>> Michael
>>>>>
>>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>
>>>>> Thanks Matthias.
>>>>> Sure, will correct it right away.
>>>>>
>>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>> wrote:
>>>>>
>>>>> Thanks for preparing this page!
>>>>>
>>>>> About terminology:
>>>>>
>>>>> You introduce the term "event time" -- but we should call this
>>>>> "stream
>>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>>> this
>>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>>
>>>>> Does this make sense to you?
>>>>>
>>>>>
>>>>>
>>>>> -Matthias
>>>>>
>>>>>
>>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>
>>>>> Thanks Ewen.
>>>>>
>>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>>
>>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>
>>>>>
>>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>>> public.
>>>>>
>>>>> --
>>>>> Arun Mathew
>>>>>
>>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>>>>> wrote:
>>>>>
>>>>>     Arun,
>>>>>
>>>>>     I've given you permission to edit the wiki. Let me know if
>>>>> you run
>>>>>
>>>>> into any
>>>>>
>>>>>     issues.
>>>>>
>>>>>     -Ewen
>>>>>
>>>>>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co rp.jp> <am...@yahoo-corp.jp>
>>>>>
>>>>> wrote:
>>>>>
>>>>>
>>>>>     > Thanks Michal. I don’t have the access yet [arunmathew88].
>>>>> Should I
>>>>>
>>>>> be
>>>>>
>>>>>     > sending a separate mail for this?
>>>>>     >
>>>>>     > I thought one of the person following this thread would be
>>>>> able to
>>>>>
>>>>> give me
>>>>>
>>>>>     > access.
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>>>>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>     > *Date: *Friday, April 7, 2017 at 17:16
>>>>>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>>>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>> semantics
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > Hi Arun,
>>>>>     >
>>>>>     > I was thinking along the same lines as you, listing the use
>>>>> cases
>>>>>
>>>>> on the
>>>>>
>>>>>     > wiki, but didn't find time to get around doing that yet.
>>>>>     > Don't mind if you do it if you have access now.
>>>>>     > I was thinking it would be nice if, once we have the use
>>>>> cases
>>>>>
>>>>> listed,
>>>>>
>>>>>     > people could use likes to up-vote the use cases similar to
>>>>> what
>>>>>
>>>>> they're
>>>>>
>>>>>     > working on.
>>>>>     >
>>>>>     > I should have a bit more time to action this in the next
>>>>> few days,
>>>>>
>>>>> but
>>>>>
>>>>>     > happy for you to do it if you can beat me to it ;-)
>>>>>     >
>>>>>     > Cheers,
>>>>>     > Michal
>>>>>     >
>>>>>     > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>     >
>>>>>     > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > Of course. I was thinking of a subpage where people can
>>>>>
>>>>> collaborate.
>>>>>
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > Will do as per Michael’s suggestion.
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > Regards,
>>>>>     >
>>>>>     > Arun Mathew
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>>>> <
>>>>>
>>>>> matthias@confluent.io> wrote:
>>>>>
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >     Please share your Wiki-ID and a committer can give you
>>>>> write
>>>>>
>>>>> access.
>>>>>
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >     Btw: as you did not initiate the KIP, you should not
>>>>> change the
>>>>>
>>>>> KIP
>>>>>
>>>>>     >
>>>>>     >     without the permission of the original author -- in
>>>>> this case
>>>>>
>>>>> Michael.
>>>>>
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >     So you might also just share your thought over the
>>>>> mailing list
>>>>>
>>>>> and
>>>>>
>>>>>     >
>>>>>     >     Michael can update the KIP page. Or, as an alternative,
>>>>> just
>>>>>
>>>>> create a
>>>>>
>>>>>     >
>>>>>     >     subpage for the KIP page.
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >     @Michael: WDYT?
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >     -Matthias
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>     >
>>>>>     >     > Hi Jay,
>>>>>     >
>>>>>     >     >           Thanks for the advise, I would like to list
>>>>> down
>>>>>
>>>>> the use cases as
>>>>>
>>>>>     >
>>>>>     >     > per your suggestion. But it seems I don't have write
>>>>>
>>>>> permission to the
>>>>>
>>>>>     >
>>>>>     >     > Apache Kafka Confluent Space. Whom shall I request
>>>>> for it?
>>>>>     >
>>>>>     >     >
>>>>>     >
>>>>>     >     > Regarding your last question. We are using a patch in
>>>>> our
>>>>>
>>>>> production system
>>>>>
>>>>>     >
>>>>>     >     > which does exactly this.
>>>>>     >
>>>>>     >     > We window by the event time, but trigger punctuate in
>>>>>
>>>>> <punctuate interval>
>>>>>
>>>>>     >
>>>>>     >     > duration of system time, in the absence of an event
>>>>> crossing
>>>>>
>>>>> the punctuate
>>>>>
>>>>>     >
>>>>>     >     > event time.
>>>>>     >
>>>>>     >     >
>>>>>     >
>>>>>     >     > We are using Kafka Stream for our Audit Trail, where
>>>>> we need
>>>>>
>>>>> to output the
>>>>>
>>>>>     >
>>>>>     >     > event counts on each topic on each cluster aggregated
>>>>> over a
>>>>>
>>>>> 1 minute
>>>>>
>>>>>     >
>>>>>     >     > window. We have to use event time to be able to cross
>>>>> check
>>>>>
>>>>> the counts. But
>>>>>
>>>>>     >
>>>>>     >     > we need to trigger punctuate [aggregate event pushes]
>>>>> by
>>>>>
>>>>> system time in the
>>>>>
>>>>>     >
>>>>>     >     > absence of events. Otherwise the event counts for
>>>>> unexpired
>>>>>
>>>>> windows would
>>>>>
>>>>>     >
>>>>>     >     > be 0 which is bad.
>>>>>     >
>>>>>     >     >
>>>>>     >
>>>>>     >     > "Maybe a hybrid solution works: I window by event
>>>>> time but
>>>>>
>>>>> trigger results
>>>>>
>>>>>     >
>>>>>     >     > by system time for windows that have updated? Not
>>>>> really sure
>>>>>
>>>>> the details
>>>>>
>>>>>     >
>>>>>     >     > of making that work. Does that work? Are there
>>>>> concrete
>>>>>
>>>>> examples where you
>>>>>
>>>>>     >
>>>>>     >     > actually want the current behavior?"
>>>>>     >
>>>>>     >     >
>>>>>     >
>>>>>     >     > --
>>>>>     >
>>>>>     >     > With Regards,
>>>>>     >
>>>>>     >     >
>>>>>     >
>>>>>     >     > Arun Mathew
>>>>>     >
>>>>>     >     > Yahoo! JAPAN Corporation
>>>>>     >
>>>>>     >     >
>>>>>     >
>>>>>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>>
>>>>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>>>>
>>>>>     >
>>>>>     >     >
>>>>>     >
>>>>>     >     >> Hi Jay,
>>>>>     >
>>>>>     >     >>
>>>>>     >
>>>>>     >     >> The hybrid solution is exactly what I expect and
>>>>> need for
>>>>>
>>>>> our use cases
>>>>>
>>>>>     >
>>>>>     >     >> when dealing with telecom data.
>>>>>     >
>>>>>     >     >>
>>>>>     >
>>>>>     >     >> Thanks
>>>>>     >
>>>>>     >     >> Tianji
>>>>>     >
>>>>>     >     >>
>>>>>     >
>>>>>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>>
>>>>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>>>>
>>>>>     >
>>>>>     >     >>
>>>>>     >
>>>>>     >     >>> Hey guys,
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>> One thing I've always found super important for
>>>>> this kind
>>>>>
>>>>> of design work
>>>>>
>>>>>     >
>>>>>     >     >> is
>>>>>     >
>>>>>     >     >>> to do a really good job of cataloging the landscape
>>>>> of use
>>>>>
>>>>> cases and how
>>>>>
>>>>>     >
>>>>>     >     >>> prevalent each one is. By that I mean not just
>>>>> listing lots
>>>>>
>>>>> of uses, but
>>>>>
>>>>>     >
>>>>>     >     >>> also grouping them into categories that
>>>>> functionally need
>>>>>
>>>>> the same thing.
>>>>>
>>>>>     >
>>>>>     >     >>> In the absence of this it is very hard to reason
>>>>> about
>>>>>
>>>>> design proposals.
>>>>>
>>>>>     >
>>>>>     >     >>> From the proposals so far I think we have a lot of
>>>>>
>>>>> discussion around
>>>>>
>>>>>     >
>>>>>     >     >>> possible apis, but less around what the user needs
>>>>> for
>>>>>
>>>>> different use
>>>>>
>>>>>     >
>>>>>     >     >> cases
>>>>>     >
>>>>>     >     >>> and how they would implement that using the api.
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>> Here is an example:
>>>>>     >
>>>>>     >     >>> You aggregate click and impression data for a
>>>>> reddit like
>>>>>
>>>>> site. Every ten
>>>>>
>>>>>     >
>>>>>     >     >>> minutes you want to output a ranked list of the top
>>>>> 10
>>>>>
>>>>> articles ranked by
>>>>>
>>>>>     >
>>>>>     >     >>> clicks/impressions for each geographical area. I
>>>>> want to be
>>>>>
>>>>> able run this
>>>>>
>>>>>     >
>>>>>     >     >>> in steady state as well as rerun to regenerate
>>>>> results (or
>>>>>
>>>>> catch up if it
>>>>>
>>>>>     >
>>>>>     >     >>> crashes).
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>> There are a couple of tricky things that seem to
>>>>> make this
>>>>>
>>>>> hard with
>>>>>
>>>>>     >
>>>>>     >     >> either
>>>>>     >
>>>>>     >     >>> of the options proposed:
>>>>>     >
>>>>>     >     >>> 1. If I emit this data using event time I have the
>>>>> problem
>>>>>
>>>>> described
>>>>>
>>>>>     >
>>>>>     >     >> where
>>>>>     >
>>>>>     >     >>> a geographical region with no new clicks or
>>>>> impressions
>>>>>
>>>>> will fail to
>>>>>
>>>>>     >
>>>>>     >     >> output
>>>>>     >
>>>>>     >     >>> results.
>>>>>     >
>>>>>     >     >>> 2. If I emit this data using system time I have the
>>>>> problem
>>>>>
>>>>> that when
>>>>>
>>>>>     >
>>>>>     >     >>> reprocessing data my window may not be ten minutes
>>>>> but 10
>>>>>
>>>>> hours if my
>>>>>
>>>>>     >
>>>>>     >     >>> processing is very fast so it dramatically changes
>>>>> the
>>>>>
>>>>> output.
>>>>>
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>> Maybe a hybrid solution works: I window by event
>>>>> time but
>>>>>
>>>>> trigger results
>>>>>
>>>>>     >
>>>>>     >     >>> by system time for windows that have updated? Not
>>>>> really
>>>>>
>>>>> sure the details
>>>>>
>>>>>     >
>>>>>     >     >>> of making that work. Does that work? Are there
>>>>> concrete
>>>>>
>>>>> examples where
>>>>>
>>>>>     >
>>>>>     >     >> you
>>>>>     >
>>>>>     >     >>> actually want the current behavior?
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>> -Jay
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>>
>>>>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>>>>
>>>>>     >
>>>>>     >     >>> wrote:
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>>> Hi All,
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> Thanks for the KIP. We were also in need of a
>>>>> mechanism to
>>>>>
>>>>> trigger
>>>>>
>>>>>     >
>>>>>     >     >>>> punctuate in the absence of events.
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> As I described in [
>>>>>     >
>>>>>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>     >
>>>>>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>     >
>>>>>     >     >>>> plugin.system.issuetabpanels:comment-
>>>>> tabpanel#comment-
>>>>>
>>>>> 15926036
>>>>>
>>>>>     >
>>>>>     >     >>>> ],
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>>    - Our approached involved using the event time
>>>>> by
>>>>>
>>>>> default.
>>>>>
>>>>>     >
>>>>>     >     >>>>    - The method to check if there is any punctuate
>>>>> ready
>>>>>
>>>>> in the
>>>>>
>>>>>     >
>>>>>     >     >>>>    PunctuationQueue is triggered via the any event
>>>>>
>>>>> received by the
>>>>>
>>>>>     >
>>>>>     >     >> stream
>>>>>     >
>>>>>     >     >>>>    tread, or at the polling intervals in the
>>>>> absence of
>>>>>
>>>>> any events.
>>>>>
>>>>>     >
>>>>>     >     >>>>    - When we create Punctuate objects (which
>>>>> contains the
>>>>>
>>>>> next event
>>>>>
>>>>>     >
>>>>>     >     >> time
>>>>>     >
>>>>>     >     >>>>    for punctuation and interval), we also record
>>>>> the
>>>>>
>>>>> creation time
>>>>>
>>>>>     >
>>>>>     >     >>> (system
>>>>>     >
>>>>>     >     >>>>    time).
>>>>>     >
>>>>>     >     >>>>    - While checking for maturity of Punctuate
>>>>> Schedule by
>>>>>     >
>>>>>     >     >> mayBePunctuate
>>>>>     >
>>>>>     >     >>>>    method, we also check if the system clock has
>>>>> elapsed
>>>>>
>>>>> the punctuate
>>>>>
>>>>>     >
>>>>>     >     >>>>    interval since the schedule creation time.
>>>>>     >
>>>>>     >     >>>>    - In the absence of any event, or in the
>>>>> absence of any
>>>>>
>>>>> event for
>>>>>
>>>>>     >
>>>>>     >     >> one
>>>>>     >
>>>>>     >     >>>>    topic in the partition group assigned to the
>>>>> stream
>>>>>
>>>>> task, the system
>>>>>
>>>>>     >
>>>>>     >     >>>> time
>>>>>     >
>>>>>     >     >>>>    will elapse the interval and we trigger a
>>>>> punctuate
>>>>>
>>>>> using the
>>>>>
>>>>>     >
>>>>>     >     >> expected
>>>>>     >
>>>>>     >     >>>>    punctuation event time.
>>>>>     >
>>>>>     >     >>>>    - we then create the next punctuation schedule
>>>>> as
>>>>>
>>>>> punctuation event
>>>>>
>>>>>     >
>>>>>     >     >>> time
>>>>>     >
>>>>>     >     >>>>    + punctuation interval, [again recording the
>>>>> system
>>>>>
>>>>> time of creation
>>>>>
>>>>>     >
>>>>>     >     >>> of
>>>>>     >
>>>>>     >     >>>> the
>>>>>     >
>>>>>     >     >>>>    schedule].
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>> approach
>>>>>
>>>>> has pros and
>>>>>
>>>>>     >
>>>>>     >     >>>> cons.
>>>>>     >
>>>>>     >     >>>> Pros
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>>    - Punctuates will happen in <punctuate
>>>>> interval> time
>>>>>
>>>>> duration at
>>>>>
>>>>>     >
>>>>>     >     >> max
>>>>>     >
>>>>>     >     >>> in
>>>>>     >
>>>>>     >     >>>>    terms of system time.
>>>>>     >
>>>>>     >     >>>>    - The semantics as a whole continues to revolve
>>>>> around
>>>>>
>>>>> event time.
>>>>>
>>>>>     >
>>>>>     >     >>>>    - We can use the old data [old timestamps] to
>>>>> rerun any
>>>>>
>>>>> experiments
>>>>>
>>>>>     >
>>>>>     >     >> or
>>>>>     >
>>>>>     >     >>>>    tests.
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> Cons
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>>    - In case the  <punctuate interval> is not a
>>>>> time
>>>>>
>>>>> duration [say
>>>>>
>>>>>     >
>>>>>     >     >>> logical
>>>>>     >
>>>>>     >     >>>>    time/event count], then the approach might not
>>>>> be
>>>>>
>>>>> meaningful.
>>>>>
>>>>>     >
>>>>>     >     >>>>    - In case there is a case where we have to wait
>>>>> for an
>>>>>
>>>>> actual event
>>>>>
>>>>>     >
>>>>>     >     >>> from
>>>>>     >
>>>>>     >     >>>>    a low event rate partition in the partition
>>>>> group, this
>>>>>
>>>>> approach
>>>>>
>>>>>     >
>>>>>     >     >> will
>>>>>     >
>>>>>     >     >>>> jump
>>>>>     >
>>>>>     >     >>>>    the gun.
>>>>>     >
>>>>>     >     >>>>    - in case the event processing cannot catch up
>>>>> with the
>>>>>
>>>>> event rate
>>>>>
>>>>>     >
>>>>>     >     >> and
>>>>>     >
>>>>>     >     >>>>    the expected timestamp events gets queued for
>>>>> long
>>>>>
>>>>> time, this
>>>>>
>>>>>     >
>>>>>     >     >> approach
>>>>>     >
>>>>>     >     >>>>    might jump the gun.
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> I believe the above approach and discussion goes
>>>>> close to
>>>>>
>>>>> the approach
>>>>>
>>>>>     >
>>>>>     >     >> A.
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> -----------
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> I like the idea of having an even count based
>>>>> punctuate.
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> -----------
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> I agree with the discussion around approach C,
>>>>> that we
>>>>>
>>>>> should provide
>>>>>
>>>>>     >
>>>>>     >     >> the
>>>>>     >
>>>>>     >     >>>> user with the option to choose system time or
>>>>> event time
>>>>>
>>>>> based
>>>>>
>>>>>     >
>>>>>     >     >>> punctuates.
>>>>>     >
>>>>>     >     >>>> But I believe that the user predominantly wants to
>>>>> use
>>>>>
>>>>> event time while
>>>>>
>>>>>     >
>>>>>     >     >>> not
>>>>>     >
>>>>>     >     >>>> missing out on regular punctuates due to event
>>>>> delays or
>>>>>
>>>>> event
>>>>>
>>>>>     >
>>>>>     >     >> absences.
>>>>>     >
>>>>>     >     >>>> Hence a complex punctuate option as Matthias
>>>>> mentioned
>>>>>
>>>>> (quoted below)
>>>>>
>>>>>     >
>>>>>     >     >>> would
>>>>>     >
>>>>>     >     >>>> be most apt.
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> "- We might want to add "complex" schedules later
>>>>> on
>>>>>
>>>>> (like, punctuate
>>>>>
>>>>>     >
>>>>>     >     >> on
>>>>>     >
>>>>>     >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>> time
>>>>>
>>>>> whatever comes
>>>>>
>>>>>     >
>>>>>     >     >>>> first)."
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> -----------
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> I think I read somewhere that Kafka Streams
>>>>> started with
>>>>>
>>>>> System Time as
>>>>>
>>>>>     >
>>>>>     >     >>> the
>>>>>     >
>>>>>     >     >>>> punctuation standard, but was later changed to
>>>>> Event Time.
>>>>>
>>>>> I guess
>>>>>
>>>>>     >
>>>>>     >     >> there
>>>>>     >
>>>>>     >     >>>> would be some good reason behind it. As Kafka
>>>>> Streams want
>>>>>
>>>>> to evolve
>>>>>
>>>>>     >
>>>>>     >     >> more
>>>>>     >
>>>>>     >     >>>> on the Stream Processing front, I believe the
>>>>> emphasis on
>>>>>
>>>>> event time
>>>>>
>>>>>     >
>>>>>     >     >>> would
>>>>>     >
>>>>>     >     >>>> remain quite strong.
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> With Regards,
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> Arun Mathew
>>>>>     >
>>>>>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>>
>>>>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>>>>
>>>>>     >
>>>>>     >     >> wrote:
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>>> Yeah I like PuncutationType much better; I just
>>>>> threw
>>>>>
>>>>> Time out there
>>>>>
>>>>>     >
>>>>>     >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>> still
>>>>>
>>>>> think it's
>>>>>
>>>>>     >
>>>>>     >     >>>>> worth considering what this buys us over an
>>>>> additional
>>>>>
>>>>> callback. I
>>>>>
>>>>>     >
>>>>>     >     >>>>> foresee a number of punctuate implementations
>>>>> following
>>>>>
>>>>> this pattern:
>>>>>
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>> public void punctuate(PunctuationType type) {
>>>>>     >
>>>>>     >     >>>>>     switch (type) {
>>>>>     >
>>>>>     >     >>>>>         case EVENT_TIME:
>>>>>     >
>>>>>     >     >>>>>             methodA();
>>>>>     >
>>>>>     >     >>>>>             break;
>>>>>     >
>>>>>     >     >>>>>         case SYSTEM_TIME:
>>>>>     >
>>>>>     >     >>>>>             methodB();
>>>>>     >
>>>>>     >     >>>>>             break;
>>>>>     >
>>>>>     >     >>>>>     }
>>>>>     >
>>>>>     >     >>>>> }
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>> I guess one advantage of this approach is we
>>>>> could add
>>>>>
>>>>> additional
>>>>>
>>>>>     >
>>>>>     >     >>>>> punctuation types later in a backwards compatible
>>>>> way
>>>>>
>>>>> (like event
>>>>>
>>>>>     >
>>>>>     >     >> count
>>>>>     >
>>>>>     >     >>>>> as you mentioned).
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>> -Tommy
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>> Sax wrote:
>>>>>     >
>>>>>     >     >>>>>> That sounds promising.
>>>>>     >
>>>>>     >     >>>>>>
>>>>>     >
>>>>>     >     >>>>>> I am just wondering if `Time` is the best name.
>>>>> Maybe we
>>>>>
>>>>> want to
>>>>>
>>>>>     >
>>>>>     >     >> add
>>>>>     >
>>>>>     >     >>>>>> other non-time based punctuations at some point
>>>>> later. I
>>>>>
>>>>> would
>>>>>
>>>>>     >
>>>>>     >     >>>>>> suggest
>>>>>     >
>>>>>     >     >>>>>>
>>>>>     >
>>>>>     >     >>>>>> enum PunctuationType {
>>>>>     >
>>>>>     >     >>>>>>   EVENT_TIME,
>>>>>     >
>>>>>     >     >>>>>>   SYSTEM_TIME,
>>>>>     >
>>>>>     >     >>>>>> }
>>>>>     >
>>>>>     >     >>>>>>
>>>>>     >
>>>>>     >     >>>>>> or similar. Just to keep the door open -- it's
>>>>> easier to
>>>>>
>>>>> add new
>>>>>
>>>>>     >
>>>>>     >     >>>>>> stuff
>>>>>     >
>>>>>     >     >>>>>> if the name is more generic.
>>>>>     >
>>>>>     >     >>>>>>
>>>>>     >
>>>>>     >     >>>>>>
>>>>>     >
>>>>>     >     >>>>>> -Matthias
>>>>>     >
>>>>>     >     >>>>>>
>>>>>     >
>>>>>     >     >>>>>>
>>>>>     >
>>>>>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>> I agree that the framework providing and
>>>>> managing the
>>>>>
>>>>> notion of
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> stream
>>>>>     >
>>>>>     >     >>>>>>> time is valuable and not something we would
>>>>> want to
>>>>>
>>>>> delegate to
>>>>>
>>>>>     >
>>>>>     >     >> the
>>>>>     >
>>>>>     >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>> separate
>>>>>
>>>>> callback
>>>>>
>>>>>     >
>>>>>     >     >> (option
>>>>>     >
>>>>>     >     >>>>>>> C)
>>>>>     >
>>>>>     >     >>>>>>> is that messy (it could just be a default
>>>>> method with
>>>>>
>>>>> an empty
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> implementation), but if we wanted a single API
>>>>> to
>>>>>
>>>>> handle both
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> cases,
>>>>>     >
>>>>>     >     >>>>>>> how about something like the following?
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>> enum Time {
>>>>>     >
>>>>>     >     >>>>>>>    STREAM,
>>>>>     >
>>>>>     >     >>>>>>>    CLOCK
>>>>>     >
>>>>>     >     >>>>>>> }
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>> Then on ProcessorContext:
>>>>>     >
>>>>>     >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>> We could
>>>>>
>>>>> allow
>>>>>
>>>>>     >
>>>>>     >     >> this
>>>>>     >
>>>>>     >     >>>>>>> to
>>>>>     >
>>>>>     >     >>>>>>> be called once for each value of time to mix
>>>>>
>>>>> approaches.
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>> Then the Processor API becomes:
>>>>>     >
>>>>>     >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>>
>>>>> schedule resulted
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> in
>>>>>     >
>>>>>     >     >>>>>>> this call.
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>> Thoughts?
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>> Sax
>>>>>
>>>>> wrote:
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> I was thinking about the four options you
>>>>> proposed in
>>>>>
>>>>> more
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> details
>>>>>     >
>>>>>     >     >>>>>>>> and
>>>>>     >
>>>>>     >     >>>>>>>> this are my thoughts:
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> (A) You argue, that users can still
>>>>> "punctuate" on
>>>>>
>>>>> event-time
>>>>>
>>>>>     >
>>>>>     >     >> via
>>>>>     >
>>>>>     >     >>>>>>>> process(), but I am not sure if this is
>>>>> possible.
>>>>>
>>>>> Note, that
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> users
>>>>>     >
>>>>>     >     >>>>>>>> only
>>>>>     >
>>>>>     >     >>>>>>>> get record timestamps via context.timestamp().
>>>>> Thus,
>>>>>
>>>>> users
>>>>>
>>>>>     >
>>>>>     >     >> would
>>>>>     >
>>>>>     >     >>>>>>>> need
>>>>>     >
>>>>>     >     >>>>>>>> to
>>>>>     >
>>>>>     >     >>>>>>>> track the time progress per partition (based
>>>>> on the
>>>>>
>>>>> partitions
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> they
>>>>>     >
>>>>>     >     >>>>>>>> obverse via context.partition(). (This alone
>>>>> puts a
>>>>>
>>>>> huge burden
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> on
>>>>>     >
>>>>>     >     >>>>>>>> the
>>>>>     >
>>>>>     >     >>>>>>>> user by itself.) However, users are not
>>>>> notified at
>>>>>
>>>>> startup
>>>>>
>>>>>     >
>>>>>     >     >> what
>>>>>     >
>>>>>     >     >>>>>>>> partitions are assigned, and user are not
>>>>> notified
>>>>>
>>>>> when
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> partitions
>>>>>     >
>>>>>     >     >>>>>>>> get
>>>>>     >
>>>>>     >     >>>>>>>> revoked. Because this information is not
>>>>> available,
>>>>>
>>>>> it's not
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> possible
>>>>>     >
>>>>>     >     >>>>>>>> to
>>>>>     >
>>>>>     >     >>>>>>>> "manually advance" stream-time, and thus
>>>>> event-time
>>>>>
>>>>> punctuation
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> within
>>>>>     >
>>>>>     >     >>>>>>>> process() seems not to be possible -- or do
>>>>> you see a
>>>>>
>>>>> way to
>>>>>
>>>>>     >
>>>>>     >     >> get
>>>>>     >
>>>>>     >     >>>>>>>> it
>>>>>     >
>>>>>     >     >>>>>>>> done? And even if, it might still be too
>>>>> clumsy to
>>>>>
>>>>> use.
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> (B) This does not allow to mix both
>>>>> approaches, thus
>>>>>
>>>>> limiting
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> what
>>>>>     >
>>>>>     >     >>>>>>>> users
>>>>>     >
>>>>>     >     >>>>>>>> can do.
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> (C) This should give all flexibility we need.
>>>>> However,
>>>>>
>>>>> just
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> adding
>>>>>     >
>>>>>     >     >>>>>>>> one
>>>>>     >
>>>>>     >     >>>>>>>> more method seems to be a solution that is too
>>>>> simple
>>>>>
>>>>> (cf my
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> comments
>>>>>     >
>>>>>     >     >>>>>>>> below).
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>> sure how
>>>>>
>>>>> a user
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> could
>>>>>     >
>>>>>     >     >>>>>>>> enable system-time and event-time punctuation
>>>>> in
>>>>>
>>>>> parallel.
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> Overall options (C) seems to be the most
>>>>> promising
>>>>>
>>>>> approach to
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> me.
>>>>>     >
>>>>>     >     >>>>>>>> Because I also favor a clean API, we might
>>>>> keep
>>>>>
>>>>> current
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> punctuate()
>>>>>     >
>>>>>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>> at some
>>>>>
>>>>> later
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> point
>>>>>     >
>>>>>     >     >>>>>>>> when
>>>>>     >
>>>>>     >     >>>>>>>> people use the "new punctuate API".
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> Couple of follow up questions:
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> - I am wondering, if we should have two
>>>>> callback
>>>>>
>>>>> methods or
>>>>>
>>>>>     >
>>>>>     >     >> just
>>>>>     >
>>>>>     >     >>>>>>>> one
>>>>>     >
>>>>>     >     >>>>>>>> (ie, a unified for system and event time
>>>>> punctuation
>>>>>
>>>>> or one for
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> each?).
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> - If we have one, how can the user figure out,
>>>>> which
>>>>>
>>>>> condition
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> did
>>>>>     >
>>>>>     >     >>>>>>>> trigger?
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> - How would the API look like, for registering
>>>>>
>>>>> different
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> punctuate
>>>>>     >
>>>>>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> - We might want to add "complex" schedules
>>>>> later on
>>>>>
>>>>> (like,
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> punctuate
>>>>>     >
>>>>>     >     >>>>>>>> on
>>>>>     >
>>>>>     >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>> system-time
>>>>>
>>>>> whatever
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> comes
>>>>>     >
>>>>>     >     >>>>>>>> first). I don't say we should add this right
>>>>> away, but
>>>>>
>>>>> we might
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> want
>>>>>     >
>>>>>     >     >>>>>>>> to
>>>>>     >
>>>>>     >     >>>>>>>> define the API in a way, that it allows
>>>>> extensions
>>>>>
>>>>> like this
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> later
>>>>>     >
>>>>>     >     >>>>>>>> on,
>>>>>     >
>>>>>     >     >>>>>>>> without redesigning the API (ie, the API
>>>>> should be
>>>>>
>>>>> designed
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> extensible)
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> - Did you ever consider count-based
>>>>> punctuation?
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> I understand, that you would like to solve a
>>>>> simple
>>>>>
>>>>> problem,
>>>>>
>>>>>     >
>>>>>     >     >> but
>>>>>     >
>>>>>     >     >>>>>>>> we
>>>>>     >
>>>>>     >     >>>>>>>> learned from the past, that just "adding some
>>>>> API"
>>>>>
>>>>> quickly
>>>>>
>>>>>     >
>>>>>     >     >> leads
>>>>>     >
>>>>>     >     >>>>>>>> to a
>>>>>     >
>>>>>     >     >>>>>>>> not very well defined API that needs time
>>>>> consuming
>>>>>
>>>>> clean up
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> later on
>>>>>     >
>>>>>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>> holistic
>>>>>     >
>>>>>     >     >>>>>>>> punctuation
>>>>>     >
>>>>>     >     >>>>>>>> KIP
>>>>>     >
>>>>>     >     >>>>>>>> with this from the beginning on to avoid later
>>>>> painful
>>>>>     >
>>>>>     >     >> redesign.
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> -Matthias
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> Thanks Thomas,
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> I'm also wary of changing the existing
>>>>> semantics of
>>>>>     >
>>>>>     >     >> punctuate,
>>>>>     >
>>>>>     >     >>>>>>>>> for
>>>>>     >
>>>>>     >     >>>>>>>>> backward compatibility reasons, although I
>>>>> like the
>>>>>     >
>>>>>     >     >> conceptual
>>>>>     >
>>>>>     >     >>>>>>>>> simplicity of that option.
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>> a way,
>>>>>
>>>>> uglier.
>>>>>
>>>>>     >
>>>>>     >     >> I
>>>>>     >
>>>>>     >     >>>>>>>>> added
>>>>>     >
>>>>>     >     >>>>>>>>> this to the KIP now as option (C).
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>> more
>>>>>
>>>>> flexible,
>>>>>
>>>>>     >
>>>>>     >     >> as
>>>>>     >
>>>>>     >     >>>>>>>>> it
>>>>>     >
>>>>>     >     >>>>>>>>> allows
>>>>>     >
>>>>>     >     >>>>>>>>> you to return any value, you're not limited
>>>>> to event
>>>>>
>>>>> time or
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> system
>>>>>     >
>>>>>     >     >>>>>>>>> time
>>>>>     >
>>>>>     >     >>>>>>>>> (although I don't see an actual use case
>>>>> where you
>>>>>
>>>>> might need
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> anything
>>>>>     >
>>>>>     >     >>>>>>>>> else then those two). Hence I also proposed
>>>>> the
>>>>>
>>>>> option to
>>>>>
>>>>>     >
>>>>>     >     >> allow
>>>>>     >
>>>>>     >     >>>>>>>>> users
>>>>>     >
>>>>>     >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>> for
>>>>>
>>>>> them given
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> the
>>>>>     >
>>>>>     >     >>>>>>>>> presence or absence of messages, much like
>>>>> they can
>>>>>
>>>>> decide
>>>>>
>>>>>     >
>>>>>     >     >> what
>>>>>     >
>>>>>     >     >>>>>>>>> msg
>>>>>     >
>>>>>     >     >>>>>>>>> time
>>>>>     >
>>>>>     >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>> What do
>>>>>
>>>>> you
>>>>>
>>>>>     >
>>>>>     >     >> think
>>>>>     >
>>>>>     >     >>>>>>>>> about
>>>>>     >
>>>>>     >     >>>>>>>>> that? This is probably most flexible but also
>>>>> most
>>>>>     >
>>>>>     >     >> complicated.
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> All comments appreciated.
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> Cheers,
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> Michal
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>     >
>>>>>     >     >>>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>> Although I fully agree we need a way to
>>>>> trigger
>>>>>
>>>>> periodic
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>> processing
>>>>>     >
>>>>>     >     >>>>>>>>>> that is independent from whether and when
>>>>> messages
>>>>>
>>>>> arrive,
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>> I'm
>>>>>     >
>>>>>     >     >>>>>>>>>> not sure
>>>>>     >
>>>>>     >     >>>>>>>>>> I like the idea of changing the existing
>>>>> semantics
>>>>>
>>>>> across
>>>>>
>>>>>     >
>>>>>     >     >> the
>>>>>     >
>>>>>     >     >>>>>>>>>> board.
>>>>>     >
>>>>>     >     >>>>>>>>>> What if we added an additional callback to
>>>>> Processor
>>>>>
>>>>> that
>>>>>
>>>>>     >
>>>>>     >     >> can
>>>>>     >
>>>>>     >     >>>>>>>>>> be
>>>>>     >
>>>>>     >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>> always
>>>>>
>>>>> called at
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>> fixed, wall
>>>>>     >
>>>>>     >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>> have to
>>>>>
>>>>> give
>>>>>
>>>>>     >
>>>>>     >     >> up
>>>>>     >
>>>>>     >     >>>>>>>>>> the
>>>>>     >
>>>>>     >     >>>>>>>>>> notion
>>>>>     >
>>>>>     >     >>>>>>>>>> of stream time to be able to do periodic
>>>>> processing.
>>>>>     >
>>>>>     >     >>>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>> Borowiecki
>>>>>
>>>>> wrote:
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>> Hi all,
>>>>>     >
>>>>>     >     >>>>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>>
>>>>> punctuate
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>> semantics
>>>>>     >
>>>>>     >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>>
>>>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/ confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>
>>>>>     >
>>>>>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>>
>>>>> 138%
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>> 3A+C
>>>>>     >
>>>>>     >     >>>>>>>>>>> hange+
>>>>>     >
>>>>>     >     >>>>>>>>>>> punctuate+semantics>
>>>>>     >
>>>>>     >     >>>>>>>>>>> .
>>>>>     >
>>>>>     >     >>>>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>> Appreciating there can be different views
>>>>> on
>>>>>
>>>>> system-time
>>>>>
>>>>>     >
>>>>>     >     >> vs
>>>>>     >
>>>>>     >     >>>>>>>>>>> event-
>>>>>     >
>>>>>     >     >>>>>>>>>>> time
>>>>>     >
>>>>>     >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>> case and
>>>>>
>>>>> the
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>> importance of
>>>>>     >
>>>>>     >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>> I've
>>>>>
>>>>> left it
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>> quite
>>>>>     >
>>>>>     >     >>>>>>>>>>> open
>>>>>     >
>>>>>     >     >>>>>>>>>>> and
>>>>>     >
>>>>>     >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>>
>>>>> progresses.
>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>>>>> Thanks,
>>>>>     >
>>>>>     >     >>>>>>>>>>> Michal
>>>>>     >
>>>>>     >     >>>>>>> --
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>     Tommy Becker
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>     Senior Software Engineer
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>     tivo.com
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>> ________________________________
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>>>> This email and any attachments may contain
>>>>> confidential
>>>>>
>>>>> and
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> privileged material for the sole use of the
>>>>> intended
>>>>>
>>>>> recipient.
>>>>>
>>>>>     >
>>>>>     >     >> Any
>>>>>     >
>>>>>     >     >>>>>>> review, copying, or distribution of this email
>>>>> (or any
>>>>>     >
>>>>>     >     >> attachments)
>>>>>     >
>>>>>     >     >>>>>>> by others is prohibited. If you are not the
>>>>> intended
>>>>>
>>>>> recipient,
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> please contact the sender immediately and
>>>>> permanently
>>>>>
>>>>> delete this
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> email and any attachments. No employee or agent
>>>>> of TiVo
>>>>>
>>>>> Inc. is
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> authorized to conclude any binding agreement on
>>>>> behalf
>>>>>
>>>>> of TiVo
>>>>>
>>>>>     >
>>>>>     >     >> Inc.
>>>>>     >
>>>>>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>> only be
>>>>>
>>>>> made by a
>>>>>
>>>>>     >
>>>>>     >     >>>>>>> signed written agreement.
>>>>>     >
>>>>>     >     >>>>>>>
>>>>>     >
>>>>>     >     >>>>> --
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>>     Tommy Becker
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>>     Senior Software Engineer
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>>     tivo.com
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>> ________________________________
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>> This email and any attachments may contain
>>>>> confidential
>>>>>
>>>>> and
>>>>>
>>>>>     >
>>>>>     >     >> privileged
>>>>>     >
>>>>>     >     >>>>> material for the sole use of the intended
>>>>> recipient. Any
>>>>>
>>>>> review,
>>>>>
>>>>>     >
>>>>>     >     >>> copying,
>>>>>     >
>>>>>     >     >>>>> or distribution of this email (or any
>>>>> attachments) by
>>>>>
>>>>> others is
>>>>>
>>>>>     >
>>>>>     >     >>>> prohibited.
>>>>>     >
>>>>>     >     >>>>> If you are not the intended recipient, please
>>>>> contact the
>>>>>
>>>>> sender
>>>>>
>>>>>     >
>>>>>     >     >>>>> immediately and permanently delete this email and
>>>>> any
>>>>>
>>>>> attachments. No
>>>>>
>>>>>     >
>>>>>     >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>> conclude
>>>>>
>>>>> any binding
>>>>>
>>>>>     >
>>>>>     >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>> Binding
>>>>>
>>>>> agreements with
>>>>>
>>>>>     >
>>>>>     >     >> TiVo
>>>>>     >
>>>>>     >     >>>>> Inc. may only be made by a signed written
>>>>> agreement.
>>>>>     >
>>>>>     >     >>>>>
>>>>>     >
>>>>>     >     >>>>
>>>>>     >
>>>>>     >     >>>
>>>>>     >
>>>>>     >     >>
>>>>>     >
>>>>>     >     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > --
>>>>>     >
>>>>>     > <http://www.openbet.com/> <http://www.openbet.com/>
>>>>>
>>>>>     >
>>>>>     > *Michal Borowiecki*
>>>>>     >
>>>>>     > *Senior Software Engineer L4*
>>>>>     >
>>>>>     > *T: *
>>>>>     >
>>>>>     > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>>     >
>>>>>     > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > *E: *
>>>>>     >
>>>>>     > michal.borowiecki@openbet.com
>>>>>     >
>>>>>     > *W: *
>>>>>     >
>>>>>     > www.openbet.com
>>>>>     >
>>>>>     > *OpenBet Ltd*
>>>>>     >
>>>>>     > Chiswick Park Building 9
>>>>>     >
>>>>>     > 566 Chiswick High Rd
>>>>>     >
>>>>>     > London
>>>>>     >
>>>>>     > W4 5XT
>>>>>     >
>>>>>     > UK
>>>>>     >
>>>>>     > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>     > This message is confidential and intended only for the
>>>>> addressee.
>>>>>
>>>>> If you
>>>>>
>>>>>     > have received this message in error, please immediately
>>>>> notify the
>>>>>     > postmaster@openbet.com and delete it from your system as
>>>>> well as
>>>>>
>>>>> any
>>>>>
>>>>>     > copies. The content of e-mails as well as traffic data may
>>>>> be
>>>>>
>>>>> monitored by
>>>>>
>>>>>     > OpenBet for employment and security purposes. To protect
>>>>> the
>>>>>
>>>>> environment
>>>>>
>>>>>     > please do not print this e-mail unless necessary. OpenBet
>>>>> Ltd.
>>>>>
>>>>> Registered
>>>>>
>>>>>     > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>> London,
>>>>>
>>>>> W4 5XT,
>>>>>
>>>>>     > United Kingdom. A company registered in England and Wales.
>>>>>
>>>>> Registered no.
>>>>>
>>>>>     > 3134634. VAT no. GB927523612
>>>>>     >
>>>>>     >
>>>>>     >
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>>
>>>>>
>>>>>     Tommy Becker
>>>>>
>>>>>     Senior Software Engineer
>>>>>
>>>>>     O +1 919.460.4747 <%28919%29%20460-4747>
>>>>>
>>>>>
>>>>>     tivo.com
>>>>>
>>>>>
>>>>>
>>>>> ________________________________
>>>>>
>>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>>
>>>>>
>>>>> --
>>>>> <http://www.openbet.com/> Michal Borowiecki
>>>>> Senior Software Engineer L4
>>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>>
>>>>>
>>>>>
>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	 
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If
>> you have received this message in error, please immediately notify the
>> postmaster@openbet.com <ma...@openbet.com> and delete it
>> from your system as well as any copies. The content of e-mails as well
>> as traffic data may be monitored by OpenBet for employment and
>> security purposes. To protect the environment please do not print this
>> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
>> company registered in England and Wales. Registered no. 3134634. VAT
>> no. GB927523612
>>
> 
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
> 
> 	
> 	+44 203 249 8448
> 
> 	
> 	 
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
> 
> 	
> 	OpenBet Ltd
> 
> 	Chiswick Park Building 9
> 
> 	566 Chiswick High Rd
> 
> 	London
> 
> 	W4 5XT
> 
> 	UK
> 
> 	
> <https://www.openbet.com/email_promo>
> 
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com <ma...@openbet.com> and delete it
> from your system as well as any copies. The content of e-mails as well
> as traffic data may be monitored by OpenBet for employment and security
> purposes. To protect the environment please do not print this e-mail
> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> registered in England and Wales. Registered no. 3134634. VAT no.
> GB927523612
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi all,

Looking for feedback on the functional interface approach Damian 
proposed. What do people think?

Further on the semantics of triggering punctuate though:

I ran through the 2 use cases that Arun had kindly put on the wiki 
(https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases) 
in my head and on a whiteboard and I can't find a better solution than 
the "hybrid" approach he had proposed.

I see how in theory this could be implemented on top of the 2 punctuate 
callbacks with the 2 different PunctuationTypes (one stream-time based, 
the other system-time based) but it would be a much more complicated 
scheme and I don't want to suggest that.

However, to add to the hybrid algorithm proposed, I suggest one 
parameter to that: a tolerance period, expressed in milliseconds 
system-time, after which the punctuation will be invoked in case the 
stream-time advance hasn't triggered it within the requested interval 
since the last invocation of punctuate (as recorded in system-time)

This would allow a user-defined tolerance for late arriving events. The 
trade off would be left for the user to decide: regular punctuation in 
the case of absence of events vs allowing for records arriving late or 
some build-up due to processing not catching up with the event rate.
In the one extreme, this tolerance could be set to infinity, turning 
hybrid into simply stream-time based punctuate, like we have now. In the 
other extreme, the tolerance could be set to 0, resulting in a 
system-time upper bound on the effective punctuation interval.

Given the above, I don't see a reason any more for a separate 
system-time based punctuation. The "hybrid" approach with 0ms tolerance 
would under normal operation trigger at regular intervals wrt the 
system-time, except in cases of re-play/catch-up, where the stream time 
advances faster than system time. In these cases punctuate would happen 
more often than the specified interval wrt system time. However, the 
use-cases that need system-time punctuations (that I've seen at least) 
really only have a need for an upper bound on punctuation delay but 
don't need a lower bound.

To that effect I'd propose the api to be as follows, on ProcessorContext:

schedule(Punctuator callback, long interval, long toleranceIterval); // schedules punctuate at stream-time intervals with a system-time upper bound of (interval+toleranceInterval)

schedule(Punctuator callback, long interval); // schedules punctuate at stream-time intervals without an system-time upper bound - this is equivalent to current stream-time based punctuate

Punctuation is triggered when either:
- the stream time advances past the (stream time of the previous 
punctuation) + interval;
- or (iff the toleranceInterval is set) when the system time advances 
past the (system time of the previous punctuation) + interval + 
toleranceInterval

In either case:
- we trigger punctuate passing as the argument the stream time at which 
the current punctuation was meant to happen
- next punctuate is scheduled at (stream time at which the current 
punctuation was meant to happen) + interval

It may appear complicated at first but I do think these semantics will 
still be more understandable to users than having 2 separate punctuation 
schedules/callbacks with different PunctuationTypes.



PS. Having re-read this, maybe the following alternative would be easier 
to understand (WDYT?):

schedule(Punctuator callback, long streamTimeInterval, long systemTimeUpperBound); // schedules punctuate at stream-time intervals with a system-time upper bound - systemTimeUpperBound must be no less than streamTimeInterval

schedule(Punctuator callback, long streamTimeInterval); // schedules punctuate at stream-time intervals without a system-time upper bound - this is equivalent to current stream-time based punctuate

Punctuation is triggered when either:
- the stream time advances past the (stream time of the previous 
punctuation) + streamTimeInterval;
- or (iff systemTimeUpperBound is set) when the system time advances 
past the (system time of the previous punctuation) + systemTimeUpperBound

Awaiting comments.

Thanks,
Michal

On 21/04/17 16:56, Michal Borowiecki wrote:
>
> Yes, that's what I meant. Just wanted to highlight we'd deprecate it 
> in favour of something that doesn't return a record. Not a problem though.
>
>
> On 21/04/17 16:32, Damian Guy wrote:
>> Thanks Michal,
>> I agree Transformer.punctuate should also be void, but we can deprecate
>> that too in favor of the new interface.
>>
>> Thanks for the javadoc PR!
>>
>> Cheers,
>> Damian
>>
>> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
>> michal.borowiecki@openbet.com> wrote:
>>
>>> Yes, that looks better to me.
>>>
>>> Note that punctuate on Transformer is currently returning a record, but I
>>> think it's ok to have all output records be sent via
>>> ProcessorContext.forward, which has to be used anyway if you want to send
>>> multiple records from one invocation of punctuate.
>>>
>>> This way it's consistent between Processor and Transformer.
>>>
>>>
>>> BTW, looking at this I found a glitch in the javadoc and put a comment
>>> there:
>>>
>>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>>
>>> and PR:https://github.com/apache/kafka/pull/2884
>>>
>>> Cheers,
>>>
>>> Michal
>>> On 20/04/17 18:55, Damian Guy wrote:
>>>
>>> Hi Michal,
>>>
>>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>>> the API.
>>> 1. deprecate the punctuate method on Processor
>>> 2. create a new Functional Interface just for Punctuation, something like:
>>> interface Punctuator {
>>>      void punctuate(long timestamp)
>>> }
>>> 3. add a new schedule function to ProcessorContext: schedule(long
>>> interval, PunctuationType type, Punctuator callback)
>>> 4. deprecate the existing schedule function
>>>
>>> Thoughts?
>>>
>>> Thanks,
>>> Damian
>>>
>>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>>> michal.borowiecki@openbet.com> wrote:
>>>
>>>> Hi Thomas,
>>>>
>>>> I would say our use cases fall in the same category as yours.
>>>>
>>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>>
>>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>>> of design:
>>>>
>>>> Incoming messages carry a number of fields, including a timestamp.
>>>>
>>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>>> by the timestamp input field (Y) and some other input field (Z).
>>>>
>>>> Since the output field X is derived in some non-trivial way, we don't
>>>> want to force the logic onto downstream apps. Instead we want to calculate
>>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>>> timestamp in Y is reached (wall clock time) and send a message if it
>>>> changed (I say "if" because the derived field (X) is also conditional on
>>>> another input field Z).
>>>>
>>>> So we have kv stores with the records and an additional kv store with
>>>> timestamp->id mapping which act like an index where we periodically do a
>>>> ranged query.
>>>>
>>>> Initially we naively tried doing it in punctuate which of course didn't
>>>> work when there were no regular msgs on the input topic.
>>>> Since this was before 0.10.1 and state stores weren't query-able from
>>>> outside we created a "ticker" that produced msgs once per second onto
>>>> another topic and fed it into the same topology to trigger punctuate.
>>>> This didn't work either, which was much more surprising to us at the
>>>> time, because it was not obvious at all that punctuate is only triggered if
>>>> *all* input partitions receive messages regularly.
>>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>>> is consumed by the second topology and is its only input topic. There's a
>>>> transformer on that topic which populates and updates the time-based
>>>> indexes and polls them from punctuate. If the time in the timestamp
>>>> elapsed, the record id is sent to the main transformer, which
>>>> updates/deletes the record from the main kv store and forwards the
>>>> transformed record to the output topic.
>>>>
>>>> To me this setup feels horrendously complicated for what it does.
>>>>
>>>> We could incrementally improve on this since 0.10.1 to poll the
>>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>>> However, the ticks don't feel so hacky when you realise they give you
>>>> some hypothetical benefits in predictability. You can reprocess the
>>>> messages in a reproducible manner, since the topologies use event-time,
>>>> just that the event time is simply the wall-clock time fed into a topic by
>>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>>> reprocessing).
>>>> To make that work though, we would have to have the stream time advance
>>>> based on the presence of msgs on the "tick" topic, regardless of the
>>>> presence of messages on the other input topic.
>>>>
>>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>>> and the hybrid would work to simplify this a lot.
>>>>
>>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>>> we can achieve it using session windows instead. I'll keep you posted if we
>>>> have to go with punctuate there too.
>>>>
>>>> Thanks,
>>>> Michal
>>>>
>>>>
>>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>>
>>>> Here's an example that we currently have.  We have a streams processor
>>>> that does a transform from one topic into another. One of the fields in
>>>> the source topic record is an expiration time, and one of the functions
>>>> of the processor is to ensure that expired records get deleted promptly
>>>> after that time passes (typically days or weeks after the message was
>>>> originally produced). To do that, the processor keeps a state store of
>>>> keys and expiration times, iterates that store in punctuate(), and
>>>> emits delete (null) records for expired items. This needs to happen at
>>>> some minimum interval regardless of the incoming message rate of the
>>>> source topic.
>>>>
>>>> In this scenario, the expiration of records is the primary function of
>>>> punctuate, and therefore the key requirement is that the wall-clock
>>>> measured time between punctuate calls have some upper-bound. So a pure
>>>> wall-clock based schedule would be fine for our needs. But the proposed
>>>> "hybrid" system would also be acceptable if that satisfies a broader
>>>> range of use-cases.
>>>>
>>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>>
>>>> I apologize for the longer email below.  To my defense, it started
>>>> out much
>>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>> devil's
>>>> advocate for a number of arguments brought forth in order to help
>>>> improve
>>>> this KIP -- I am not implying I necessarily disagree with the
>>>> arguments.
>>>>
>>>> That aside, here are some further thoughts.
>>>>
>>>> First, there are (at least?) two categories for actions/behavior you
>>>> invoke
>>>> via punctuate():
>>>>
>>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>> to
>>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>>> the
>>>> impact of punctuate is typically not observable by other processing
>>>> nodes
>>>> in the topology.
>>>> 2. For controlling the emit frequency of downstream records.  Here,
>>>> the
>>>> punctuate is all about being observable by downstream processing
>>>> nodes.
>>>>
>>>> A few releases back, we introduced record caches (DSL) and state
>>>> store
>>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>> relating to
>>>> (2) where some users needed to control -- here: limit -- the
>>>> downstream
>>>> output rate of Kafka Streams because the downstream systems/apps
>>>> would not
>>>> be able to keep up with the upstream output rate (Kafka scalability >
>>>> their
>>>> scalability).  The argument for KIP-63, which notably did not
>>>> introduce a
>>>> "trigger" API, was that such an interaction with downstream systems
>>>> is an
>>>> operational concern;  it should not impact the processing *logic* of
>>>> your
>>>> application, and thus we didn't want to complicate the Kafka Streams
>>>> API,
>>>> especially not the declarative DSL, with such operational concerns.
>>>>
>>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>>> sorry, I
>>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>> observing that our conversation is moving more and more into the
>>>> direction
>>>> of explicit "triggers" because, so far, I have seen only motivations
>>>> for
>>>> use cases in category (2), but none yet for (1)?  For example, some
>>>> comments voiced here are about sth like "IF stream-time didn't
>>>> trigger
>>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>> want
>>>> this, and if so, for which use cases and benefits?  Also, on a
>>>> related
>>>> note, whatever we are discussing here will impact state store caches
>>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>>> should
>>>> clarify any such impact here.
>>>>
>>>> Switching topics slightly.
>>>>
>>>> Jay wrote:
>>>>
>>>> One thing I've always found super important for this kind of design
>>>> work
>>>> is to do a really good job of cataloging the landscape of use cases
>>>> and
>>>> how prevalent each one is.
>>>>
>>>> +1 to this, as others have already said.
>>>>
>>>> Here, let me highlight -- just in case -- that when we talked about
>>>> windowing use cases in the recent emails, the Processor API (where
>>>> `punctuate` resides) does not have any notion of windowing at
>>>> all.  If you
>>>> want to do windowing *in the Processor API*, you must do so manually
>>>> in
>>>> combination with window stores.  For this reason I'd suggest to
>>>> discuss use
>>>> cases not just in general, but also in view of how you'd do so in the
>>>> Processor API vs. in the DSL.  Right now, changing/improving
>>>> `punctuate`
>>>> does not impact the DSL at all, unless we add new functionality to
>>>> it.
>>>>
>>>> Jay wrote in his strawman example:
>>>>
>>>> You aggregate click and impression data for a reddit like site.
>>>> Every ten
>>>> minutes you want to output a ranked list of the top 10 articles
>>>> ranked by
>>>> clicks/impressions for each geographical area. I want to be able
>>>> run this
>>>> in steady state as well as rerun to regenerate results (or catch up
>>>> if it
>>>> crashes).
>>>>
>>>> This is a good example for more than the obvious reason:  In KIP-63,
>>>> we
>>>> argued that the reason for saying "every ten minutes" above is not
>>>> necessarily about because you want to output data *exactly* after ten
>>>> minutes, but that you want to perform an aggregation based on 10-
>>>> minute
>>>> windows of input data; i.e., the point is about specifying the input
>>>> for
>>>> your aggregation, not or less about when the results of the
>>>> aggregation
>>>> should be send downstream.  To take an extreme example, you could
>>>> disable
>>>> record caches and let your app output a downstream update for every
>>>> incoming input record.  If the last input record was from at minute 7
>>>> of 10
>>>> (for a 10-min window), then what your app would output at minute 10
>>>> would
>>>> be identical to what it had already emitted at minute 7 earlier
>>>> anyways.
>>>> This is particularly true when we take late-arriving data into
>>>> account:  if
>>>> a late record arrived at minute 13, your app would (by default) send
>>>> a new
>>>> update downstream, even though the "original" 10 minutes have already
>>>> passed.
>>>>
>>>> Jay wrote...:
>>>>
>>>> There are a couple of tricky things that seem to make this hard
>>>> with
>>>>
>>>> either
>>>>
>>>> of the options proposed:
>>>> 1. If I emit this data using event time I have the problem
>>>> described where
>>>> a geographical region with no new clicks or impressions will fail
>>>> to
>>>>
>>>> output
>>>>
>>>> results.
>>>>
>>>> ...and Arun Mathew wrote:
>>>>
>>>>
>>>> We window by the event time, but trigger punctuate in <punctuate
>>>> interval>
>>>> duration of system time, in the absence of an event crossing the
>>>> punctuate
>>>> event time.
>>>>
>>>> So, given what I wrote above about the status quo and what you can
>>>> already
>>>> do with it, is the concern that the state store cache doesn't give
>>>> you
>>>> *direct* control over "forcing an output after no later than X
>>>> seconds [of
>>>> processing-time]" but only indirect control through a cache
>>>> size?  (Note
>>>> that I am not dismissing the claims why this might be helpful.)
>>>>
>>>> Arun Mathew wrote:
>>>>
>>>> We are using Kafka Stream for our Audit Trail, where we need to
>>>> output the
>>>> event counts on each topic on each cluster aggregated over a 1
>>>> minute
>>>> window. We have to use event time to be able to cross check the
>>>> counts.
>>>>
>>>> But
>>>>
>>>> we need to trigger punctuate [aggregate event pushes] by system
>>>> time in
>>>>
>>>> the
>>>>
>>>> absence of events. Otherwise the event counts for unexpired windows
>>>> would
>>>> be 0 which is bad.
>>>>
>>>> Isn't the latter -- "count would be 0" -- the problem between the
>>>> absence
>>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>>> Scala
>>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>>> root
>>>> cause that the downstream system interprets the absence of output in
>>>> a
>>>> particular way ("No output after 1 minute = I consider the output to
>>>> be
>>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>>> possible)
>>>> to correctly handle the difference between absence of output vs.
>>>> output of
>>>> 0.  I am not implying that we shouldn't care about such a use case,
>>>> but
>>>> want to understand the motivation better. :-)
>>>>
>>>> Also, to add some perspective, in some related discussions we talked
>>>> about
>>>> how a Kafka Streams application should not worry or not be coupled
>>>> unnecessarily with such interpretation specifics in a downstream
>>>> system's
>>>> behavior.  After all, tomorrow your app's output might be consumed by
>>>> more
>>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>>> than
>>>> Kafka Streams might be the best tool to link the universes of Kafka
>>>> and
>>>> downstream systems, including helping to reconcile the differences in
>>>> how
>>>> these systems interpret changes, updates, late-arriving data,
>>>> etc.  Kafka
>>>> Connect would allow you to decouple the Kafka Streams app's logical
>>>> processing from the specifics of downstream systems, thanks to
>>>> specific
>>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>> this
>>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>> but
>>>> it's currently awkward to use Connect for this", this might be a
>>>> problem we
>>>> can solve, too.)
>>>>
>>>> Switching topics slightly again.
>>>>
>>>> Thomas wrote:
>>>>
>>>> I'm not entirely convinced that a separate callback (option C)
>>>> is that messy (it could just be a default method with an empty
>>>> implementation), but if we wanted a single API to handle both
>>>> cases,
>>>> how about something like the following?
>>>>
>>>> enum Time {
>>>>     STREAM,
>>>>     CLOCK
>>>> }
>>>>
>>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>> then
>>>> whatever the user is doing inside this method is a black box to Kafka
>>>> Streams (similar to how we have no idea what the user does inside a
>>>> `foreach` -- if the function passed to `foreach` writes to external
>>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>>> won't
>>>> know, for example, if the stream-time action has a smaller "trigger"
>>>> frequency than the processing-time action.  Or, we won't know whether
>>>> the
>>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>> minute of
>>>> stream-time or 1-minute of processing-time, whichever comes
>>>> first").  That
>>>> said, I am not certain yet whether we would need such knowledge
>>>> because,
>>>> when using the Processor API, most of the work and decisions must be
>>>> done
>>>> by the user anyways.  It would matter though if the concept of
>>>> "triggers"
>>>> were to bubble up into the DSL because in the DSL the management of
>>>> windowing, window stores, etc. must be done automatically by Kafka
>>>> Streams.
>>>>
>>>> [In any case, btw, we have the corner case where the user configured
>>>> the
>>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>>> extractor), at which point both punctuate variants are based on the
>>>> same
>>>> time semantics / timeline.]
>>>>
>>>> Again, I apologize for the wall of text.  Congratulations if you made
>>>> it
>>>> this far. :-)
>>>>
>>>> More than happy to hear your thoughts!
>>>> Michael
>>>>
>>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew<ar...@gmail.com>  <ar...@gmail.com>
>>>> wrote:
>>>>
>>>>
>>>> Thanks Matthias.
>>>> Sure, will correct it right away.
>>>>
>>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax"<ma...@confluent.io>  <ma...@confluent.io>
>>>> wrote:
>>>>
>>>> Thanks for preparing this page!
>>>>
>>>> About terminology:
>>>>
>>>> You introduce the term "event time" -- but we should call this
>>>> "stream
>>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>>> this
>>>> could be event time, ingestion time, or processing/wall-clock time.
>>>>
>>>> Does this make sense to you?
>>>>
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>
>>>> Thanks Ewen.
>>>>
>>>> @Michal, @all, I have created a child page to start the Use Cases
>>>>
>>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>
>>>>
>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>> public.
>>>>
>>>> --
>>>> Arun Mathew
>>>>
>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava"<ew...@confluent.io>  <ew...@confluent.io>
>>>> wrote:
>>>>
>>>>      Arun,
>>>>
>>>>      I've given you permission to edit the wiki. Let me know if
>>>> you run
>>>>
>>>> into any
>>>>
>>>>      issues.
>>>>
>>>>      -Ewen
>>>>
>>>>      On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew<amathew@yahoo-co rp.jp>  <am...@yahoo-corp.jp>
>>>>
>>>> wrote:
>>>>
>>>>
>>>>      > Thanks Michal. I don\u2019t have the access yet [arunmathew88].
>>>> Should I
>>>>
>>>> be
>>>>
>>>>      > sending a separate mail for this?
>>>>      >
>>>>      > I thought one of the person following this thread would be
>>>> able to
>>>>
>>>> give me
>>>>
>>>>      > access.
>>>>      >
>>>>      >
>>>>      >
>>>>      > *From: *Michal Borowiecki<mi...@openbet.com>  <mi...@openbet.com>
>>>>      > *Reply-To: *"dev@kafka.apache.org"  <de...@kafka.apache.org>  <de...@kafka.apache.org>  <de...@kafka.apache.org>
>>>>      > *Date: *Friday, April 7, 2017 at 17:16
>>>>      > *To: *"dev@kafka.apache.org"  <de...@kafka.apache.org>  <de...@kafka.apache.org>  <de...@kafka.apache.org>
>>>>      > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>> semantics
>>>>      >
>>>>      >
>>>>      >
>>>>      > Hi Arun,
>>>>      >
>>>>      > I was thinking along the same lines as you, listing the use
>>>> cases
>>>>
>>>> on the
>>>>
>>>>      > wiki, but didn't find time to get around doing that yet.
>>>>      > Don't mind if you do it if you have access now.
>>>>      > I was thinking it would be nice if, once we have the use
>>>> cases
>>>>
>>>> listed,
>>>>
>>>>      > people could use likes to up-vote the use cases similar to
>>>> what
>>>>
>>>> they're
>>>>
>>>>      > working on.
>>>>      >
>>>>      > I should have a bit more time to action this in the next
>>>> few days,
>>>>
>>>> but
>>>>
>>>>      > happy for you to do it if you can beat me to it ;-)
>>>>      >
>>>>      > Cheers,
>>>>      > Michal
>>>>      >
>>>>      > On 07/04/17 04:39, Arun Mathew wrote:
>>>>      >
>>>>      > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>      >
>>>>      >
>>>>      >
>>>>      > Of course. I was thinking of a subpage where people can
>>>>
>>>> collaborate.
>>>>
>>>>      >
>>>>      >
>>>>      >
>>>>      > Will do as per Michael\u2019s suggestion.
>>>>      >
>>>>      >
>>>>      >
>>>>      > Regards,
>>>>      >
>>>>      > Arun Mathew
>>>>      >
>>>>      >
>>>>      >
>>>>      > On 4/7/17, 12:30, "Matthias J. Sax"<ma...@confluent.io>  <ma...@confluent.io>
>>>> <
>>>>
>>>> matthias@confluent.io> wrote:
>>>>
>>>>      >
>>>>      >
>>>>      >
>>>>      >     Please share your Wiki-ID and a committer can give you
>>>> write
>>>>
>>>> access.
>>>>
>>>>      >
>>>>      >
>>>>      >
>>>>      >     Btw: as you did not initiate the KIP, you should not
>>>> change the
>>>>
>>>> KIP
>>>>
>>>>      >
>>>>      >     without the permission of the original author -- in
>>>> this case
>>>>
>>>> Michael.
>>>>
>>>>      >
>>>>      >
>>>>      >
>>>>      >     So you might also just share your thought over the
>>>> mailing list
>>>>
>>>> and
>>>>
>>>>      >
>>>>      >     Michael can update the KIP page. Or, as an alternative,
>>>> just
>>>>
>>>> create a
>>>>
>>>>      >
>>>>      >     subpage for the KIP page.
>>>>      >
>>>>      >
>>>>      >
>>>>      >     @Michael: WDYT?
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >     -Matthias
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>      >
>>>>      >     > Hi Jay,
>>>>      >
>>>>      >     >           Thanks for the advise, I would like to list
>>>> down
>>>>
>>>> the use cases as
>>>>
>>>>      >
>>>>      >     > per your suggestion. But it seems I don't have write
>>>>
>>>> permission to the
>>>>
>>>>      >
>>>>      >     > Apache Kafka Confluent Space. Whom shall I request
>>>> for it?
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > Regarding your last question. We are using a patch in
>>>> our
>>>>
>>>> production system
>>>>
>>>>      >
>>>>      >     > which does exactly this.
>>>>      >
>>>>      >     > We window by the event time, but trigger punctuate in
>>>>
>>>> <punctuate interval>
>>>>
>>>>      >
>>>>      >     > duration of system time, in the absence of an event
>>>> crossing
>>>>
>>>> the punctuate
>>>>
>>>>      >
>>>>      >     > event time.
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > We are using Kafka Stream for our Audit Trail, where
>>>> we need
>>>>
>>>> to output the
>>>>
>>>>      >
>>>>      >     > event counts on each topic on each cluster aggregated
>>>> over a
>>>>
>>>> 1 minute
>>>>
>>>>      >
>>>>      >     > window. We have to use event time to be able to cross
>>>> check
>>>>
>>>> the counts. But
>>>>
>>>>      >
>>>>      >     > we need to trigger punctuate [aggregate event pushes]
>>>> by
>>>>
>>>> system time in the
>>>>
>>>>      >
>>>>      >     > absence of events. Otherwise the event counts for
>>>> unexpired
>>>>
>>>> windows would
>>>>
>>>>      >
>>>>      >     > be 0 which is bad.
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > "Maybe a hybrid solution works: I window by event
>>>> time but
>>>>
>>>> trigger results
>>>>
>>>>      >
>>>>      >     > by system time for windows that have updated? Not
>>>> really sure
>>>>
>>>> the details
>>>>
>>>>      >
>>>>      >     > of making that work. Does that work? Are there
>>>> concrete
>>>>
>>>> examples where you
>>>>
>>>>      >
>>>>      >     > actually want the current behavior?"
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > --
>>>>      >
>>>>      >     > With Regards,
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > Arun Mathew
>>>>      >
>>>>      >     > Yahoo! JAPAN Corporation
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>
>>>> skyahead@gmail.com><sk...@gmail.com>  <sk...@gmail.com>  wrote:
>>>>
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     >> Hi Jay,
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >> The hybrid solution is exactly what I expect and
>>>> need for
>>>>
>>>> our use cases
>>>>
>>>>      >
>>>>      >     >> when dealing with telecom data.
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >> Thanks
>>>>      >
>>>>      >     >> Tianji
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>
>>>> jay@confluent.io><ja...@confluent.io>  <ja...@confluent.io>  wrote:
>>>>
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >>> Hey guys,
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> One thing I've always found super important for
>>>> this kind
>>>>
>>>> of design work
>>>>
>>>>      >
>>>>      >     >> is
>>>>      >
>>>>      >     >>> to do a really good job of cataloging the landscape
>>>> of use
>>>>
>>>> cases and how
>>>>
>>>>      >
>>>>      >     >>> prevalent each one is. By that I mean not just
>>>> listing lots
>>>>
>>>> of uses, but
>>>>
>>>>      >
>>>>      >     >>> also grouping them into categories that
>>>> functionally need
>>>>
>>>> the same thing.
>>>>
>>>>      >
>>>>      >     >>> In the absence of this it is very hard to reason
>>>> about
>>>>
>>>> design proposals.
>>>>
>>>>      >
>>>>      >     >>> From the proposals so far I think we have a lot of
>>>>
>>>> discussion around
>>>>
>>>>      >
>>>>      >     >>> possible apis, but less around what the user needs
>>>> for
>>>>
>>>> different use
>>>>
>>>>      >
>>>>      >     >> cases
>>>>      >
>>>>      >     >>> and how they would implement that using the api.
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> Here is an example:
>>>>      >
>>>>      >     >>> You aggregate click and impression data for a
>>>> reddit like
>>>>
>>>> site. Every ten
>>>>
>>>>      >
>>>>      >     >>> minutes you want to output a ranked list of the top
>>>> 10
>>>>
>>>> articles ranked by
>>>>
>>>>      >
>>>>      >     >>> clicks/impressions for each geographical area. I
>>>> want to be
>>>>
>>>> able run this
>>>>
>>>>      >
>>>>      >     >>> in steady state as well as rerun to regenerate
>>>> results (or
>>>>
>>>> catch up if it
>>>>
>>>>      >
>>>>      >     >>> crashes).
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> There are a couple of tricky things that seem to
>>>> make this
>>>>
>>>> hard with
>>>>
>>>>      >
>>>>      >     >> either
>>>>      >
>>>>      >     >>> of the options proposed:
>>>>      >
>>>>      >     >>> 1. If I emit this data using event time I have the
>>>> problem
>>>>
>>>> described
>>>>
>>>>      >
>>>>      >     >> where
>>>>      >
>>>>      >     >>> a geographical region with no new clicks or
>>>> impressions
>>>>
>>>> will fail to
>>>>
>>>>      >
>>>>      >     >> output
>>>>      >
>>>>      >     >>> results.
>>>>      >
>>>>      >     >>> 2. If I emit this data using system time I have the
>>>> problem
>>>>
>>>> that when
>>>>
>>>>      >
>>>>      >     >>> reprocessing data my window may not be ten minutes
>>>> but 10
>>>>
>>>> hours if my
>>>>
>>>>      >
>>>>      >     >>> processing is very fast so it dramatically changes
>>>> the
>>>>
>>>> output.
>>>>
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> Maybe a hybrid solution works: I window by event
>>>> time but
>>>>
>>>> trigger results
>>>>
>>>>      >
>>>>      >     >>> by system time for windows that have updated? Not
>>>> really
>>>>
>>>> sure the details
>>>>
>>>>      >
>>>>      >     >>> of making that work. Does that work? Are there
>>>> concrete
>>>>
>>>> examples where
>>>>
>>>>      >
>>>>      >     >> you
>>>>      >
>>>>      >     >>> actually want the current behavior?
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> -Jay
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>
>>>> arunmathew88@gmail.com><ar...@gmail.com>  <ar...@gmail.com>
>>>>
>>>>      >
>>>>      >     >>> wrote:
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>>> Hi All,
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> Thanks for the KIP. We were also in need of a
>>>> mechanism to
>>>>
>>>> trigger
>>>>
>>>>      >
>>>>      >     >>>> punctuate in the absence of events.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> As I described in [
>>>>      >
>>>>      >     >>>>https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>      >
>>>>      >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>      >
>>>>      >     >>>> plugin.system.issuetabpanels:comment-
>>>> tabpanel#comment-
>>>>
>>>> 15926036
>>>>
>>>>      >
>>>>      >     >>>> ],
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>    - Our approached involved using the event time
>>>> by
>>>>
>>>> default.
>>>>
>>>>      >
>>>>      >     >>>>    - The method to check if there is any punctuate
>>>> ready
>>>>
>>>> in the
>>>>
>>>>      >
>>>>      >     >>>>    PunctuationQueue is triggered via the any event
>>>>
>>>> received by the
>>>>
>>>>      >
>>>>      >     >> stream
>>>>      >
>>>>      >     >>>>    tread, or at the polling intervals in the
>>>> absence of
>>>>
>>>> any events.
>>>>
>>>>      >
>>>>      >     >>>>    - When we create Punctuate objects (which
>>>> contains the
>>>>
>>>> next event
>>>>
>>>>      >
>>>>      >     >> time
>>>>      >
>>>>      >     >>>>    for punctuation and interval), we also record
>>>> the
>>>>
>>>> creation time
>>>>
>>>>      >
>>>>      >     >>> (system
>>>>      >
>>>>      >     >>>>    time).
>>>>      >
>>>>      >     >>>>    - While checking for maturity of Punctuate
>>>> Schedule by
>>>>      >
>>>>      >     >> mayBePunctuate
>>>>      >
>>>>      >     >>>>    method, we also check if the system clock has
>>>> elapsed
>>>>
>>>> the punctuate
>>>>
>>>>      >
>>>>      >     >>>>    interval since the schedule creation time.
>>>>      >
>>>>      >     >>>>    - In the absence of any event, or in the
>>>> absence of any
>>>>
>>>> event for
>>>>
>>>>      >
>>>>      >     >> one
>>>>      >
>>>>      >     >>>>    topic in the partition group assigned to the
>>>> stream
>>>>
>>>> task, the system
>>>>
>>>>      >
>>>>      >     >>>> time
>>>>      >
>>>>      >     >>>>    will elapse the interval and we trigger a
>>>> punctuate
>>>>
>>>> using the
>>>>
>>>>      >
>>>>      >     >> expected
>>>>      >
>>>>      >     >>>>    punctuation event time.
>>>>      >
>>>>      >     >>>>    - we then create the next punctuation schedule
>>>> as
>>>>
>>>> punctuation event
>>>>
>>>>      >
>>>>      >     >>> time
>>>>      >
>>>>      >     >>>>    + punctuation interval, [again recording the
>>>> system
>>>>
>>>> time of creation
>>>>
>>>>      >
>>>>      >     >>> of
>>>>      >
>>>>      >     >>>> the
>>>>      >
>>>>      >     >>>>    schedule].
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>> approach
>>>>
>>>> has pros and
>>>>
>>>>      >
>>>>      >     >>>> cons.
>>>>      >
>>>>      >     >>>> Pros
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>    - Punctuates will happen in <punctuate
>>>> interval> time
>>>>
>>>> duration at
>>>>
>>>>      >
>>>>      >     >> max
>>>>      >
>>>>      >     >>> in
>>>>      >
>>>>      >     >>>>    terms of system time.
>>>>      >
>>>>      >     >>>>    - The semantics as a whole continues to revolve
>>>> around
>>>>
>>>> event time.
>>>>
>>>>      >
>>>>      >     >>>>    - We can use the old data [old timestamps] to
>>>> rerun any
>>>>
>>>> experiments
>>>>
>>>>      >
>>>>      >     >> or
>>>>      >
>>>>      >     >>>>    tests.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> Cons
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>    - In case the  <punctuate interval> is not a
>>>> time
>>>>
>>>> duration [say
>>>>
>>>>      >
>>>>      >     >>> logical
>>>>      >
>>>>      >     >>>>    time/event count], then the approach might not
>>>> be
>>>>
>>>> meaningful.
>>>>
>>>>      >
>>>>      >     >>>>    - In case there is a case where we have to wait
>>>> for an
>>>>
>>>> actual event
>>>>
>>>>      >
>>>>      >     >>> from
>>>>      >
>>>>      >     >>>>    a low event rate partition in the partition
>>>> group, this
>>>>
>>>> approach
>>>>
>>>>      >
>>>>      >     >> will
>>>>      >
>>>>      >     >>>> jump
>>>>      >
>>>>      >     >>>>    the gun.
>>>>      >
>>>>      >     >>>>    - in case the event processing cannot catch up
>>>> with the
>>>>
>>>> event rate
>>>>
>>>>      >
>>>>      >     >> and
>>>>      >
>>>>      >     >>>>    the expected timestamp events gets queued for
>>>> long
>>>>
>>>> time, this
>>>>
>>>>      >
>>>>      >     >> approach
>>>>      >
>>>>      >     >>>>    might jump the gun.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> I believe the above approach and discussion goes
>>>> close to
>>>>
>>>> the approach
>>>>
>>>>      >
>>>>      >     >> A.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> -----------
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> I like the idea of having an even count based
>>>> punctuate.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> -----------
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> I agree with the discussion around approach C,
>>>> that we
>>>>
>>>> should provide
>>>>
>>>>      >
>>>>      >     >> the
>>>>      >
>>>>      >     >>>> user with the option to choose system time or
>>>> event time
>>>>
>>>> based
>>>>
>>>>      >
>>>>      >     >>> punctuates.
>>>>      >
>>>>      >     >>>> But I believe that the user predominantly wants to
>>>> use
>>>>
>>>> event time while
>>>>
>>>>      >
>>>>      >     >>> not
>>>>      >
>>>>      >     >>>> missing out on regular punctuates due to event
>>>> delays or
>>>>
>>>> event
>>>>
>>>>      >
>>>>      >     >> absences.
>>>>      >
>>>>      >     >>>> Hence a complex punctuate option as Matthias
>>>> mentioned
>>>>
>>>> (quoted below)
>>>>
>>>>      >
>>>>      >     >>> would
>>>>      >
>>>>      >     >>>> be most apt.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> "- We might want to add "complex" schedules later
>>>> on
>>>>
>>>> (like, punctuate
>>>>
>>>>      >
>>>>      >     >> on
>>>>      >
>>>>      >     >>>> every 10 seconds event-time or 60 seconds system-
>>>> time
>>>>
>>>> whatever comes
>>>>
>>>>      >
>>>>      >     >>>> first)."
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> -----------
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> I think I read somewhere that Kafka Streams
>>>> started with
>>>>
>>>> System Time as
>>>>
>>>>      >
>>>>      >     >>> the
>>>>      >
>>>>      >     >>>> punctuation standard, but was later changed to
>>>> Event Time.
>>>>
>>>> I guess
>>>>
>>>>      >
>>>>      >     >> there
>>>>      >
>>>>      >     >>>> would be some good reason behind it. As Kafka
>>>> Streams want
>>>>
>>>> to evolve
>>>>
>>>>      >
>>>>      >     >> more
>>>>      >
>>>>      >     >>>> on the Stream Processing front, I believe the
>>>> emphasis on
>>>>
>>>> event time
>>>>
>>>>      >
>>>>      >     >>> would
>>>>      >
>>>>      >     >>>> remain quite strong.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> With Regards,
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> Arun Mathew
>>>>      >
>>>>      >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>
>>>> tobecker@tivo.com><to...@tivo.com>  <to...@tivo.com>
>>>>
>>>>      >
>>>>      >     >> wrote:
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>> Yeah I like PuncutationType much better; I just
>>>> threw
>>>>
>>>> Time out there
>>>>
>>>>      >
>>>>      >     >>>>> more as a strawman than an actual suggestion ;) I
>>>> still
>>>>
>>>> think it's
>>>>
>>>>      >
>>>>      >     >>>>> worth considering what this buys us over an
>>>> additional
>>>>
>>>> callback. I
>>>>
>>>>      >
>>>>      >     >>>>> foresee a number of punctuate implementations
>>>> following
>>>>
>>>> this pattern:
>>>>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> public void punctuate(PunctuationType type) {
>>>>      >
>>>>      >     >>>>>     switch (type) {
>>>>      >
>>>>      >     >>>>>         case EVENT_TIME:
>>>>      >
>>>>      >     >>>>>             methodA();
>>>>      >
>>>>      >     >>>>>             break;
>>>>      >
>>>>      >     >>>>>         case SYSTEM_TIME:
>>>>      >
>>>>      >     >>>>>             methodB();
>>>>      >
>>>>      >     >>>>>             break;
>>>>      >
>>>>      >     >>>>>     }
>>>>      >
>>>>      >     >>>>> }
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> I guess one advantage of this approach is we
>>>> could add
>>>>
>>>> additional
>>>>
>>>>      >
>>>>      >     >>>>> punctuation types later in a backwards compatible
>>>> way
>>>>
>>>> (like event
>>>>
>>>>      >
>>>>      >     >> count
>>>>      >
>>>>      >     >>>>> as you mentioned).
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> -Tommy
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>> Sax wrote:
>>>>      >
>>>>      >     >>>>>> That sounds promising.
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> I am just wondering if `Time` is the best name.
>>>> Maybe we
>>>>
>>>> want to
>>>>
>>>>      >
>>>>      >     >> add
>>>>      >
>>>>      >     >>>>>> other non-time based punctuations at some point
>>>> later. I
>>>>
>>>> would
>>>>
>>>>      >
>>>>      >     >>>>>> suggest
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> enum PunctuationType {
>>>>      >
>>>>      >     >>>>>>   EVENT_TIME,
>>>>      >
>>>>      >     >>>>>>   SYSTEM_TIME,
>>>>      >
>>>>      >     >>>>>> }
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> or similar. Just to keep the door open -- it's
>>>> easier to
>>>>
>>>> add new
>>>>
>>>>      >
>>>>      >     >>>>>> stuff
>>>>      >
>>>>      >     >>>>>> if the name is more generic.
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> -Matthias
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> I agree that the framework providing and
>>>> managing the
>>>>
>>>> notion of
>>>>
>>>>      >
>>>>      >     >>>>>>> stream
>>>>      >
>>>>      >     >>>>>>> time is valuable and not something we would
>>>> want to
>>>>
>>>> delegate to
>>>>
>>>>      >
>>>>      >     >> the
>>>>      >
>>>>      >     >>>>>>> tasks. I'm not entirely convinced that a
>>>> separate
>>>>
>>>> callback
>>>>
>>>>      >
>>>>      >     >> (option
>>>>      >
>>>>      >     >>>>>>> C)
>>>>      >
>>>>      >     >>>>>>> is that messy (it could just be a default
>>>> method with
>>>>
>>>> an empty
>>>>
>>>>      >
>>>>      >     >>>>>>> implementation), but if we wanted a single API
>>>> to
>>>>
>>>> handle both
>>>>
>>>>      >
>>>>      >     >>>>>>> cases,
>>>>      >
>>>>      >     >>>>>>> how about something like the following?
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> enum Time {
>>>>      >
>>>>      >     >>>>>>>    STREAM,
>>>>      >
>>>>      >     >>>>>>>    CLOCK
>>>>      >
>>>>      >     >>>>>>> }
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> Then on ProcessorContext:
>>>>      >
>>>>      >     >>>>>>> context.schedule(Time time, long interval)  //
>>>> We could
>>>>
>>>> allow
>>>>
>>>>      >
>>>>      >     >> this
>>>>      >
>>>>      >     >>>>>>> to
>>>>      >
>>>>      >     >>>>>>> be called once for each value of time to mix
>>>>
>>>> approaches.
>>>>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> Then the Processor API becomes:
>>>>      >
>>>>      >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>
>>>> schedule resulted
>>>>
>>>>      >
>>>>      >     >>>>>>> in
>>>>      >
>>>>      >     >>>>>>> this call.
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> Thoughts?
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>> Sax
>>>>
>>>> wrote:
>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> I was thinking about the four options you
>>>> proposed in
>>>>
>>>> more
>>>>
>>>>      >
>>>>      >     >>>>>>>> details
>>>>      >
>>>>      >     >>>>>>>> and
>>>>      >
>>>>      >     >>>>>>>> this are my thoughts:
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> (A) You argue, that users can still
>>>> "punctuate" on
>>>>
>>>> event-time
>>>>
>>>>      >
>>>>      >     >> via
>>>>      >
>>>>      >     >>>>>>>> process(), but I am not sure if this is
>>>> possible.
>>>>
>>>> Note, that
>>>>
>>>>      >
>>>>      >     >>>>>>>> users
>>>>      >
>>>>      >     >>>>>>>> only
>>>>      >
>>>>      >     >>>>>>>> get record timestamps via context.timestamp().
>>>> Thus,
>>>>
>>>> users
>>>>
>>>>      >
>>>>      >     >> would
>>>>      >
>>>>      >     >>>>>>>> need
>>>>      >
>>>>      >     >>>>>>>> to
>>>>      >
>>>>      >     >>>>>>>> track the time progress per partition (based
>>>> on the
>>>>
>>>> partitions
>>>>
>>>>      >
>>>>      >     >>>>>>>> they
>>>>      >
>>>>      >     >>>>>>>> obverse via context.partition(). (This alone
>>>> puts a
>>>>
>>>> huge burden
>>>>
>>>>      >
>>>>      >     >>>>>>>> on
>>>>      >
>>>>      >     >>>>>>>> the
>>>>      >
>>>>      >     >>>>>>>> user by itself.) However, users are not
>>>> notified at
>>>>
>>>> startup
>>>>
>>>>      >
>>>>      >     >> what
>>>>      >
>>>>      >     >>>>>>>> partitions are assigned, and user are not
>>>> notified
>>>>
>>>> when
>>>>
>>>>      >
>>>>      >     >>>>>>>> partitions
>>>>      >
>>>>      >     >>>>>>>> get
>>>>      >
>>>>      >     >>>>>>>> revoked. Because this information is not
>>>> available,
>>>>
>>>> it's not
>>>>
>>>>      >
>>>>      >     >>>>>>>> possible
>>>>      >
>>>>      >     >>>>>>>> to
>>>>      >
>>>>      >     >>>>>>>> "manually advance" stream-time, and thus
>>>> event-time
>>>>
>>>> punctuation
>>>>
>>>>      >
>>>>      >     >>>>>>>> within
>>>>      >
>>>>      >     >>>>>>>> process() seems not to be possible -- or do
>>>> you see a
>>>>
>>>> way to
>>>>
>>>>      >
>>>>      >     >> get
>>>>      >
>>>>      >     >>>>>>>> it
>>>>      >
>>>>      >     >>>>>>>> done? And even if, it might still be too
>>>> clumsy to
>>>>
>>>> use.
>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> (B) This does not allow to mix both
>>>> approaches, thus
>>>>
>>>> limiting
>>>>
>>>>      >
>>>>      >     >>>>>>>> what
>>>>      >
>>>>      >     >>>>>>>> users
>>>>      >
>>>>      >     >>>>>>>> can do.
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> (C) This should give all flexibility we need.
>>>> However,
>>>>
>>>> just
>>>>
>>>>      >
>>>>      >     >>>>>>>> adding
>>>>      >
>>>>      >     >>>>>>>> one
>>>>      >
>>>>      >     >>>>>>>> more method seems to be a solution that is too
>>>> simple
>>>>
>>>> (cf my
>>>>
>>>>      >
>>>>      >     >>>>>>>> comments
>>>>      >
>>>>      >     >>>>>>>> below).
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>> sure how
>>>>
>>>> a user
>>>>
>>>>      >
>>>>      >     >>>>>>>> could
>>>>      >
>>>>      >     >>>>>>>> enable system-time and event-time punctuation
>>>> in
>>>>
>>>> parallel.
>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> Overall options (C) seems to be the most
>>>> promising
>>>>
>>>> approach to
>>>>
>>>>      >
>>>>      >     >>>>>>>> me.
>>>>      >
>>>>      >     >>>>>>>> Because I also favor a clean API, we might
>>>> keep
>>>>
>>>> current
>>>>
>>>>      >
>>>>      >     >>>>>>>> punctuate()
>>>>      >
>>>>      >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>> at some
>>>>
>>>> later
>>>>
>>>>      >
>>>>      >     >>>>>>>> point
>>>>      >
>>>>      >     >>>>>>>> when
>>>>      >
>>>>      >     >>>>>>>> people use the "new punctuate API".
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> Couple of follow up questions:
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - I am wondering, if we should have two
>>>> callback
>>>>
>>>> methods or
>>>>
>>>>      >
>>>>      >     >> just
>>>>      >
>>>>      >     >>>>>>>> one
>>>>      >
>>>>      >     >>>>>>>> (ie, a unified for system and event time
>>>> punctuation
>>>>
>>>> or one for
>>>>
>>>>      >
>>>>      >     >>>>>>>> each?).
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - If we have one, how can the user figure out,
>>>> which
>>>>
>>>> condition
>>>>
>>>>      >
>>>>      >     >>>>>>>> did
>>>>      >
>>>>      >     >>>>>>>> trigger?
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - How would the API look like, for registering
>>>>
>>>> different
>>>>
>>>>      >
>>>>      >     >>>>>>>> punctuate
>>>>      >
>>>>      >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - We might want to add "complex" schedules
>>>> later on
>>>>
>>>> (like,
>>>>
>>>>      >
>>>>      >     >>>>>>>> punctuate
>>>>      >
>>>>      >     >>>>>>>> on
>>>>      >
>>>>      >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>> system-time
>>>>
>>>> whatever
>>>>
>>>>      >
>>>>      >     >>>>>>>> comes
>>>>      >
>>>>      >     >>>>>>>> first). I don't say we should add this right
>>>> away, but
>>>>
>>>> we might
>>>>
>>>>      >
>>>>      >     >>>>>>>> want
>>>>      >
>>>>      >     >>>>>>>> to
>>>>      >
>>>>      >     >>>>>>>> define the API in a way, that it allows
>>>> extensions
>>>>
>>>> like this
>>>>
>>>>      >
>>>>      >     >>>>>>>> later
>>>>      >
>>>>      >     >>>>>>>> on,
>>>>      >
>>>>      >     >>>>>>>> without redesigning the API (ie, the API
>>>> should be
>>>>
>>>> designed
>>>>
>>>>      >
>>>>      >     >>>>>>>> extensible)
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - Did you ever consider count-based
>>>> punctuation?
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> I understand, that you would like to solve a
>>>> simple
>>>>
>>>> problem,
>>>>
>>>>      >
>>>>      >     >> but
>>>>      >
>>>>      >     >>>>>>>> we
>>>>      >
>>>>      >     >>>>>>>> learned from the past, that just "adding some
>>>> API"
>>>>
>>>> quickly
>>>>
>>>>      >
>>>>      >     >> leads
>>>>      >
>>>>      >     >>>>>>>> to a
>>>>      >
>>>>      >     >>>>>>>> not very well defined API that needs time
>>>> consuming
>>>>
>>>> clean up
>>>>
>>>>      >
>>>>      >     >>>>>>>> later on
>>>>      >
>>>>      >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>> holistic
>>>>      >
>>>>      >     >>>>>>>> punctuation
>>>>      >
>>>>      >     >>>>>>>> KIP
>>>>      >
>>>>      >     >>>>>>>> with this from the beginning on to avoid later
>>>> painful
>>>>      >
>>>>      >     >> redesign.
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> -Matthias
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> Thanks Thomas,
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> I'm also wary of changing the existing
>>>> semantics of
>>>>      >
>>>>      >     >> punctuate,
>>>>      >
>>>>      >     >>>>>>>>> for
>>>>      >
>>>>      >     >>>>>>>>> backward compatibility reasons, although I
>>>> like the
>>>>      >
>>>>      >     >> conceptual
>>>>      >
>>>>      >     >>>>>>>>> simplicity of that option.
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>> a way,
>>>>
>>>> uglier.
>>>>
>>>>      >
>>>>      >     >> I
>>>>      >
>>>>      >     >>>>>>>>> added
>>>>      >
>>>>      >     >>>>>>>>> this to the KIP now as option (C).
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>> more
>>>>
>>>> flexible,
>>>>
>>>>      >
>>>>      >     >> as
>>>>      >
>>>>      >     >>>>>>>>> it
>>>>      >
>>>>      >     >>>>>>>>> allows
>>>>      >
>>>>      >     >>>>>>>>> you to return any value, you're not limited
>>>> to event
>>>>
>>>> time or
>>>>
>>>>      >
>>>>      >     >>>>>>>>> system
>>>>      >
>>>>      >     >>>>>>>>> time
>>>>      >
>>>>      >     >>>>>>>>> (although I don't see an actual use case
>>>> where you
>>>>
>>>> might need
>>>>
>>>>      >
>>>>      >     >>>>>>>>> anything
>>>>      >
>>>>      >     >>>>>>>>> else then those two). Hence I also proposed
>>>> the
>>>>
>>>> option to
>>>>
>>>>      >
>>>>      >     >> allow
>>>>      >
>>>>      >     >>>>>>>>> users
>>>>      >
>>>>      >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>> for
>>>>
>>>> them given
>>>>
>>>>      >
>>>>      >     >>>>>>>>> the
>>>>      >
>>>>      >     >>>>>>>>> presence or absence of messages, much like
>>>> they can
>>>>
>>>> decide
>>>>
>>>>      >
>>>>      >     >> what
>>>>      >
>>>>      >     >>>>>>>>> msg
>>>>      >
>>>>      >     >>>>>>>>> time
>>>>      >
>>>>      >     >>>>>>>>> means for them using the TimestampExtractor.
>>>> What do
>>>>
>>>> you
>>>>
>>>>      >
>>>>      >     >> think
>>>>      >
>>>>      >     >>>>>>>>> about
>>>>      >
>>>>      >     >>>>>>>>> that? This is probably most flexible but also
>>>> most
>>>>      >
>>>>      >     >> complicated.
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> All comments appreciated.
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> Cheers,
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> Michal
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>      >
>>>>      >     >>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>> Although I fully agree we need a way to
>>>> trigger
>>>>
>>>> periodic
>>>>
>>>>      >
>>>>      >     >>>>>>>>>> processing
>>>>      >
>>>>      >     >>>>>>>>>> that is independent from whether and when
>>>> messages
>>>>
>>>> arrive,
>>>>
>>>>      >
>>>>      >     >>>>>>>>>> I'm
>>>>      >
>>>>      >     >>>>>>>>>> not sure
>>>>      >
>>>>      >     >>>>>>>>>> I like the idea of changing the existing
>>>> semantics
>>>>
>>>> across
>>>>
>>>>      >
>>>>      >     >> the
>>>>      >
>>>>      >     >>>>>>>>>> board.
>>>>      >
>>>>      >     >>>>>>>>>> What if we added an additional callback to
>>>> Processor
>>>>
>>>> that
>>>>
>>>>      >
>>>>      >     >> can
>>>>      >
>>>>      >     >>>>>>>>>> be
>>>>      >
>>>>      >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>> always
>>>>
>>>> called at
>>>>
>>>>      >
>>>>      >     >>>>>>>>>> fixed, wall
>>>>      >
>>>>      >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>> have to
>>>>
>>>> give
>>>>
>>>>      >
>>>>      >     >> up
>>>>      >
>>>>      >     >>>>>>>>>> the
>>>>      >
>>>>      >     >>>>>>>>>> notion
>>>>      >
>>>>      >     >>>>>>>>>> of stream time to be able to do periodic
>>>> processing.
>>>>      >
>>>>      >     >>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>> Borowiecki
>>>>
>>>> wrote:
>>>>
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> Hi all,
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>
>>>> punctuate
>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> semantics
>>>>      >
>>>>      >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>
>>>> confluence/display/KAFKA/KIP-<https://cwiki.apache.org/ confluence/display/KAFKA/KIP->  <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>
>>>>      >
>>>>      >     ><https://cwiki.apache.org/confluence/display/KAFKA/KI P->  <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>
>>>> 138%
>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> 3A+C
>>>>      >
>>>>      >     >>>>>>>>>>> hange+
>>>>      >
>>>>      >     >>>>>>>>>>> punctuate+semantics>
>>>>      >
>>>>      >     >>>>>>>>>>> .
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> Appreciating there can be different views
>>>> on
>>>>
>>>> system-time
>>>>
>>>>      >
>>>>      >     >> vs
>>>>      >
>>>>      >     >>>>>>>>>>> event-
>>>>      >
>>>>      >     >>>>>>>>>>> time
>>>>      >
>>>>      >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>> case and
>>>>
>>>> the
>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> importance of
>>>>      >
>>>>      >     >>>>>>>>>>> backwards compatibility of any such change,
>>>> I've
>>>>
>>>> left it
>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> quite
>>>>      >
>>>>      >     >>>>>>>>>>> open
>>>>      >
>>>>      >     >>>>>>>>>>> and
>>>>      >
>>>>      >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>
>>>> progresses.
>>>>
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> Thanks,
>>>>      >
>>>>      >     >>>>>>>>>>> Michal
>>>>      >
>>>>      >     >>>>>>> --
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>     Tommy Becker
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>     Senior Software Engineer
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>     tivo.com
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> ________________________________
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> This email and any attachments may contain
>>>> confidential
>>>>
>>>> and
>>>>
>>>>      >
>>>>      >     >>>>>>> privileged material for the sole use of the
>>>> intended
>>>>
>>>> recipient.
>>>>
>>>>      >
>>>>      >     >> Any
>>>>      >
>>>>      >     >>>>>>> review, copying, or distribution of this email
>>>> (or any
>>>>      >
>>>>      >     >> attachments)
>>>>      >
>>>>      >     >>>>>>> by others is prohibited. If you are not the
>>>> intended
>>>>
>>>> recipient,
>>>>
>>>>      >
>>>>      >     >>>>>>> please contact the sender immediately and
>>>> permanently
>>>>
>>>> delete this
>>>>
>>>>      >
>>>>      >     >>>>>>> email and any attachments. No employee or agent
>>>> of TiVo
>>>>
>>>> Inc. is
>>>>
>>>>      >
>>>>      >     >>>>>>> authorized to conclude any binding agreement on
>>>> behalf
>>>>
>>>> of TiVo
>>>>
>>>>      >
>>>>      >     >> Inc.
>>>>      >
>>>>      >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>> only be
>>>>
>>>> made by a
>>>>
>>>>      >
>>>>      >     >>>>>>> signed written agreement.
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>> --
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>     Tommy Becker
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>     Senior Software Engineer
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>     tivo.com
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> ________________________________
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> This email and any attachments may contain
>>>> confidential
>>>>
>>>> and
>>>>
>>>>      >
>>>>      >     >> privileged
>>>>      >
>>>>      >     >>>>> material for the sole use of the intended
>>>> recipient. Any
>>>>
>>>> review,
>>>>
>>>>      >
>>>>      >     >>> copying,
>>>>      >
>>>>      >     >>>>> or distribution of this email (or any
>>>> attachments) by
>>>>
>>>> others is
>>>>
>>>>      >
>>>>      >     >>>> prohibited.
>>>>      >
>>>>      >     >>>>> If you are not the intended recipient, please
>>>> contact the
>>>>
>>>> sender
>>>>
>>>>      >
>>>>      >     >>>>> immediately and permanently delete this email and
>>>> any
>>>>
>>>> attachments. No
>>>>
>>>>      >
>>>>      >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>> conclude
>>>>
>>>> any binding
>>>>
>>>>      >
>>>>      >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>> Binding
>>>>
>>>> agreements with
>>>>
>>>>      >
>>>>      >     >> TiVo
>>>>      >
>>>>      >     >>>>> Inc. may only be made by a signed written
>>>> agreement.
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      > --
>>>>      >
>>>>      ><http://www.openbet.com/>  <http://www.openbet.com/>
>>>>
>>>>      >
>>>>      > *Michal Borowiecki*
>>>>      >
>>>>      > *Senior Software Engineer L4*
>>>>      >
>>>>      > *T: *
>>>>      >
>>>>      > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>>      >
>>>>      > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>>      >
>>>>      >
>>>>      >
>>>>      > *E: *
>>>>      >
>>>>      >michal.borowiecki@openbet.com
>>>>      >
>>>>      > *W: *
>>>>      >
>>>>      >www.openbet.com
>>>>      >
>>>>      > *OpenBet Ltd*
>>>>      >
>>>>      > Chiswick Park Building 9
>>>>      >
>>>>      > 566 Chiswick High Rd
>>>>      >
>>>>      > London
>>>>      >
>>>>      > W4 5XT
>>>>      >
>>>>      > UK
>>>>      >
>>>>      ><https://www.openbet.com/email_promo>  <https://www.openbet.com/email_promo>
>>>>      >
>>>>      >
>>>>      >
>>>>      > This message is confidential and intended only for the
>>>> addressee.
>>>>
>>>> If you
>>>>
>>>>      > have received this message in error, please immediately
>>>> notify the
>>>>      >postmaster@openbet.com  and delete it from your system as
>>>> well as
>>>>
>>>> any
>>>>
>>>>      > copies. The content of e-mails as well as traffic data may
>>>> be
>>>>
>>>> monitored by
>>>>
>>>>      > OpenBet for employment and security purposes. To protect
>>>> the
>>>>
>>>> environment
>>>>
>>>>      > please do not print this e-mail unless necessary. OpenBet
>>>> Ltd.
>>>>
>>>> Registered
>>>>
>>>>      > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>> London,
>>>>
>>>> W4 5XT,
>>>>
>>>>      > United Kingdom. A company registered in England and Wales.
>>>>
>>>> Registered no.
>>>>
>>>>      > 3134634. VAT no. GB927523612
>>>>      >
>>>>      >
>>>>      >
>>>>
>>>>
>>>>
>>>> --
>>>>
>>>>
>>>>      Tommy Becker
>>>>
>>>>      Senior Software Engineer
>>>>
>>>>      O +1 919.460.4747 <%28919%29%20460-4747>
>>>>
>>>>
>>>>      tivo.com
>>>>
>>>>
>>>>
>>>> ________________________________
>>>>
>>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>>
>>>>
>>>> --
>>>> <http://www.openbet.com/>  Michal Borowiecki
>>>> Senior Software Engineer L4
>>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>>
>>>>
>>>>
>
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
>
> 	
> 	+44 203 249 8448
>
> 	
> 	
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
>
> 	
> 	OpenBet Ltd
>
> 	Chiswick Park Building 9
>
> 	566 Chiswick High Rd
>
> 	London
>
> 	W4 5XT
>
> 	UK
>
> 	
> <https://www.openbet.com/email_promo>
>
> This message is confidential and intended only for the addressee. If 
> you have received this message in error, please immediately notify the 
> postmaster@openbet.com <ma...@openbet.com> and delete it 
> from your system as well as any copies. The content of e-mails as well 
> as traffic data may be monitored by OpenBet for employment and 
> security purposes. To protect the environment please do not print this 
> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park 
> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A 
> company registered in England and Wales. Registered no. 3134634. VAT 
> no. GB927523612
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Yes, that's what I meant. Just wanted to highlight we'd deprecate it in 
favour of something that doesn't return a record. Not a problem though.


The javadoc PR passed tests on scala 2.10 and 2.11 but failed on 2.12 
although it says no tests failed Build finished. 4730 tests run, 0 
skipped, 0 failed.

https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/3086/console

> FAILURE: Build failed with an exception.
>
> * What went wrong:
> Execution failed for task ':streams:integrationTest'.
> > Process 'Gradle Test Executor 27' finished with non-zero exit value 1
Do you happen to know if this may be an intermittent issue with the 
build process?

If so, how can I re-trigger the build?


Thanks,

Michal


On 21/04/17 16:32, Damian Guy wrote:
> Thanks Michal,
> I agree Transformer.punctuate should also be void, but we can deprecate
> that too in favor of the new interface.
>
> Thanks for the javadoc PR!
>
> Cheers,
> Damian
>
> On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
> michal.borowiecki@openbet.com> wrote:
>
>> Yes, that looks better to me.
>>
>> Note that punctuate on Transformer is currently returning a record, but I
>> think it's ok to have all output records be sent via
>> ProcessorContext.forward, which has to be used anyway if you want to send
>> multiple records from one invocation of punctuate.
>>
>> This way it's consistent between Processor and Transformer.
>>
>>
>> BTW, looking at this I found a glitch in the javadoc and put a comment
>> there:
>>
>> https://github.com/apache/kafka/pull/2413/files#r112634612
>>
>> and PR: https://github.com/apache/kafka/pull/2884
>>
>> Cheers,
>>
>> Michal
>> On 20/04/17 18:55, Damian Guy wrote:
>>
>> Hi Michal,
>>
>> Thanks for the KIP. I'd like to propose a bit more of a radical change to
>> the API.
>> 1. deprecate the punctuate method on Processor
>> 2. create a new Functional Interface just for Punctuation, something like:
>> interface Punctuator {
>>      void punctuate(long timestamp)
>> }
>> 3. add a new schedule function to ProcessorContext: schedule(long
>> interval, PunctuationType type, Punctuator callback)
>> 4. deprecate the existing schedule function
>>
>> Thoughts?
>>
>> Thanks,
>> Damian
>>
>> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
>> michal.borowiecki@openbet.com> wrote:
>>
>>> Hi Thomas,
>>>
>>> I would say our use cases fall in the same category as yours.
>>>
>>> 1) One is expiry of old records, it's virtually identical to yours.
>>>
>>> 2) Second one is somewhat more convoluted but boils down to the same type
>>> of design:
>>>
>>> Incoming messages carry a number of fields, including a timestamp.
>>>
>>> Outgoing messages contain derived fields, one of them (X) is depended on
>>> by the timestamp input field (Y) and some other input field (Z).
>>>
>>> Since the output field X is derived in some non-trivial way, we don't
>>> want to force the logic onto downstream apps. Instead we want to calculate
>>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>>> timestamp in Y is reached (wall clock time) and send a message if it
>>> changed (I say "if" because the derived field (X) is also conditional on
>>> another input field Z).
>>>
>>> So we have kv stores with the records and an additional kv store with
>>> timestamp->id mapping which act like an index where we periodically do a
>>> ranged query.
>>>
>>> Initially we naively tried doing it in punctuate which of course didn't
>>> work when there were no regular msgs on the input topic.
>>> Since this was before 0.10.1 and state stores weren't query-able from
>>> outside we created a "ticker" that produced msgs once per second onto
>>> another topic and fed it into the same topology to trigger punctuate.
>>> This didn't work either, which was much more surprising to us at the
>>> time, because it was not obvious at all that punctuate is only triggered if
>>> *all* input partitions receive messages regularly.
>>> In the end we had to break this into 2 separate Kafka Streams. Main
>>> transformer doesn't use punctuate but sends values of timestamp field Y and
>>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>>> is consumed by the second topology and is its only input topic. There's a
>>> transformer on that topic which populates and updates the time-based
>>> indexes and polls them from punctuate. If the time in the timestamp
>>> elapsed, the record id is sent to the main transformer, which
>>> updates/deletes the record from the main kv store and forwards the
>>> transformed record to the output topic.
>>>
>>> To me this setup feels horrendously complicated for what it does.
>>>
>>> We could incrementally improve on this since 0.10.1 to poll the
>>> timestamp->id "index" stores from some code outside the KafkaStreams
>>> topology so that at least we wouldn't need the extra topic for "ticks".
>>> However, the ticks don't feel so hacky when you realise they give you
>>> some hypothetical benefits in predictability. You can reprocess the
>>> messages in a reproducible manner, since the topologies use event-time,
>>> just that the event time is simply the wall-clock time fed into a topic by
>>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>>> reprocessing).
>>> To make that work though, we would have to have the stream time advance
>>> based on the presence of msgs on the "tick" topic, regardless of the
>>> presence of messages on the other input topic.
>>>
>>> Same as in the expiry use case, both the wall-clock triggered punctuate
>>> and the hybrid would work to simplify this a lot.
>>>
>>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>>> we can achieve it using session windows instead. I'll keep you posted if we
>>> have to go with punctuate there too.
>>>
>>> Thanks,
>>> Michal
>>>
>>>
>>> On 11/04/17 20:52, Thomas Becker wrote:
>>>
>>> Here's an example that we currently have.  We have a streams processor
>>> that does a transform from one topic into another. One of the fields in
>>> the source topic record is an expiration time, and one of the functions
>>> of the processor is to ensure that expired records get deleted promptly
>>> after that time passes (typically days or weeks after the message was
>>> originally produced). To do that, the processor keeps a state store of
>>> keys and expiration times, iterates that store in punctuate(), and
>>> emits delete (null) records for expired items. This needs to happen at
>>> some minimum interval regardless of the incoming message rate of the
>>> source topic.
>>>
>>> In this scenario, the expiration of records is the primary function of
>>> punctuate, and therefore the key requirement is that the wall-clock
>>> measured time between punctuate calls have some upper-bound. So a pure
>>> wall-clock based schedule would be fine for our needs. But the proposed
>>> "hybrid" system would also be acceptable if that satisfies a broader
>>> range of use-cases.
>>>
>>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>
>>> I apologize for the longer email below.  To my defense, it started
>>> out much
>>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>>> devil's
>>> advocate for a number of arguments brought forth in order to help
>>> improve
>>> this KIP -- I am not implying I necessarily disagree with the
>>> arguments.
>>>
>>> That aside, here are some further thoughts.
>>>
>>> First, there are (at least?) two categories for actions/behavior you
>>> invoke
>>> via punctuate():
>>>
>>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>>> to
>>> periodically commit to a custom store, to do metrics/logging).  Here,
>>> the
>>> impact of punctuate is typically not observable by other processing
>>> nodes
>>> in the topology.
>>> 2. For controlling the emit frequency of downstream records.  Here,
>>> the
>>> punctuate is all about being observable by downstream processing
>>> nodes.
>>>
>>> A few releases back, we introduced record caches (DSL) and state
>>> store
>>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>>> relating to
>>> (2) where some users needed to control -- here: limit -- the
>>> downstream
>>> output rate of Kafka Streams because the downstream systems/apps
>>> would not
>>> be able to keep up with the upstream output rate (Kafka scalability >
>>> their
>>> scalability).  The argument for KIP-63, which notably did not
>>> introduce a
>>> "trigger" API, was that such an interaction with downstream systems
>>> is an
>>> operational concern;  it should not impact the processing *logic* of
>>> your
>>> application, and thus we didn't want to complicate the Kafka Streams
>>> API,
>>> especially not the declarative DSL, with such operational concerns.
>>>
>>> This KIP's discussion on `punctuate()` takes us back in time (<--
>>> sorry, I
>>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>> observing that our conversation is moving more and more into the
>>> direction
>>> of explicit "triggers" because, so far, I have seen only motivations
>>> for
>>> use cases in category (2), but none yet for (1)?  For example, some
>>> comments voiced here are about sth like "IF stream-time didn't
>>> trigger
>>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>>> want
>>> this, and if so, for which use cases and benefits?  Also, on a
>>> related
>>> note, whatever we are discussing here will impact state store caches
>>> (Processor API) and perhaps also impact record caches (DSL), thus we
>>> should
>>> clarify any such impact here.
>>>
>>> Switching topics slightly.
>>>
>>> Jay wrote:
>>>
>>> One thing I've always found super important for this kind of design
>>> work
>>> is to do a really good job of cataloging the landscape of use cases
>>> and
>>> how prevalent each one is.
>>>
>>> +1 to this, as others have already said.
>>>
>>> Here, let me highlight -- just in case -- that when we talked about
>>> windowing use cases in the recent emails, the Processor API (where
>>> `punctuate` resides) does not have any notion of windowing at
>>> all.  If you
>>> want to do windowing *in the Processor API*, you must do so manually
>>> in
>>> combination with window stores.  For this reason I'd suggest to
>>> discuss use
>>> cases not just in general, but also in view of how you'd do so in the
>>> Processor API vs. in the DSL.  Right now, changing/improving
>>> `punctuate`
>>> does not impact the DSL at all, unless we add new functionality to
>>> it.
>>>
>>> Jay wrote in his strawman example:
>>>
>>> You aggregate click and impression data for a reddit like site.
>>> Every ten
>>> minutes you want to output a ranked list of the top 10 articles
>>> ranked by
>>> clicks/impressions for each geographical area. I want to be able
>>> run this
>>> in steady state as well as rerun to regenerate results (or catch up
>>> if it
>>> crashes).
>>>
>>> This is a good example for more than the obvious reason:  In KIP-63,
>>> we
>>> argued that the reason for saying "every ten minutes" above is not
>>> necessarily about because you want to output data *exactly* after ten
>>> minutes, but that you want to perform an aggregation based on 10-
>>> minute
>>> windows of input data; i.e., the point is about specifying the input
>>> for
>>> your aggregation, not or less about when the results of the
>>> aggregation
>>> should be send downstream.  To take an extreme example, you could
>>> disable
>>> record caches and let your app output a downstream update for every
>>> incoming input record.  If the last input record was from at minute 7
>>> of 10
>>> (for a 10-min window), then what your app would output at minute 10
>>> would
>>> be identical to what it had already emitted at minute 7 earlier
>>> anyways.
>>> This is particularly true when we take late-arriving data into
>>> account:  if
>>> a late record arrived at minute 13, your app would (by default) send
>>> a new
>>> update downstream, even though the "original" 10 minutes have already
>>> passed.
>>>
>>> Jay wrote...:
>>>
>>> There are a couple of tricky things that seem to make this hard
>>> with
>>>
>>> either
>>>
>>> of the options proposed:
>>> 1. If I emit this data using event time I have the problem
>>> described where
>>> a geographical region with no new clicks or impressions will fail
>>> to
>>>
>>> output
>>>
>>> results.
>>>
>>> ...and Arun Mathew wrote:
>>>
>>>
>>> We window by the event time, but trigger punctuate in <punctuate
>>> interval>
>>> duration of system time, in the absence of an event crossing the
>>> punctuate
>>> event time.
>>>
>>> So, given what I wrote above about the status quo and what you can
>>> already
>>> do with it, is the concern that the state store cache doesn't give
>>> you
>>> *direct* control over "forcing an output after no later than X
>>> seconds [of
>>> processing-time]" but only indirect control through a cache
>>> size?  (Note
>>> that I am not dismissing the claims why this might be helpful.)
>>>
>>> Arun Mathew wrote:
>>>
>>> We are using Kafka Stream for our Audit Trail, where we need to
>>> output the
>>> event counts on each topic on each cluster aggregated over a 1
>>> minute
>>> window. We have to use event time to be able to cross check the
>>> counts.
>>>
>>> But
>>>
>>> we need to trigger punctuate [aggregate event pushes] by system
>>> time in
>>>
>>> the
>>>
>>> absence of events. Otherwise the event counts for unexpired windows
>>> would
>>> be 0 which is bad.
>>>
>>> Isn't the latter -- "count would be 0" -- the problem between the
>>> absence
>>> of output vs. an output of 0, similar to the use of `Option[T]` in
>>> Scala
>>> and the difference between `None` and `Some(0)`?  That is, isn't the
>>> root
>>> cause that the downstream system interprets the absence of output in
>>> a
>>> particular way ("No output after 1 minute = I consider the output to
>>> be
>>> 0.")?  Arguably, you could also adapt the downstream system (if
>>> possible)
>>> to correctly handle the difference between absence of output vs.
>>> output of
>>> 0.  I am not implying that we shouldn't care about such a use case,
>>> but
>>> want to understand the motivation better. :-)
>>>
>>> Also, to add some perspective, in some related discussions we talked
>>> about
>>> how a Kafka Streams application should not worry or not be coupled
>>> unnecessarily with such interpretation specifics in a downstream
>>> system's
>>> behavior.  After all, tomorrow your app's output might be consumed by
>>> more
>>> than just this one downstream system.  Arguably, Kafka Connect rather
>>> than
>>> Kafka Streams might be the best tool to link the universes of Kafka
>>> and
>>> downstream systems, including helping to reconcile the differences in
>>> how
>>> these systems interpret changes, updates, late-arriving data,
>>> etc.  Kafka
>>> Connect would allow you to decouple the Kafka Streams app's logical
>>> processing from the specifics of downstream systems, thanks to
>>> specific
>>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>> this
>>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>> but
>>> it's currently awkward to use Connect for this", this might be a
>>> problem we
>>> can solve, too.)
>>>
>>> Switching topics slightly again.
>>>
>>> Thomas wrote:
>>>
>>> I'm not entirely convinced that a separate callback (option C)
>>> is that messy (it could just be a default method with an empty
>>> implementation), but if we wanted a single API to handle both
>>> cases,
>>> how about something like the following?
>>>
>>> enum Time {
>>>     STREAM,
>>>     CLOCK
>>> }
>>>
>>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>> then
>>> whatever the user is doing inside this method is a black box to Kafka
>>> Streams (similar to how we have no idea what the user does inside a
>>> `foreach` -- if the function passed to `foreach` writes to external
>>> systems, then Kafka Streams is totally unaware of the fact).  We
>>> won't
>>> know, for example, if the stream-time action has a smaller "trigger"
>>> frequency than the processing-time action.  Or, we won't know whether
>>> the
>>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>>> minute of
>>> stream-time or 1-minute of processing-time, whichever comes
>>> first").  That
>>> said, I am not certain yet whether we would need such knowledge
>>> because,
>>> when using the Processor API, most of the work and decisions must be
>>> done
>>> by the user anyways.  It would matter though if the concept of
>>> "triggers"
>>> were to bubble up into the DSL because in the DSL the management of
>>> windowing, window stores, etc. must be done automatically by Kafka
>>> Streams.
>>>
>>> [In any case, btw, we have the corner case where the user configured
>>> the
>>> stream-time to be processing-time (e.g. via wall-clock timestamp
>>> extractor), at which point both punctuate variants are based on the
>>> same
>>> time semantics / timeline.]
>>>
>>> Again, I apologize for the wall of text.  Congratulations if you made
>>> it
>>> this far. :-)
>>>
>>> More than happy to hear your thoughts!
>>> Michael
>>>
>>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>>> wrote:
>>>
>>>
>>> Thanks Matthias.
>>> Sure, will correct it right away.
>>>
>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>> wrote:
>>>
>>> Thanks for preparing this page!
>>>
>>> About terminology:
>>>
>>> You introduce the term "event time" -- but we should call this
>>> "stream
>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>> this
>>> could be event time, ingestion time, or processing/wall-clock time.
>>>
>>> Does this make sense to you?
>>>
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>
>>> Thanks Ewen.
>>>
>>> @Michal, @all, I have created a child page to start the Use Cases
>>>
>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>
>>>
>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>> public.
>>>
>>> --
>>> Arun Mathew
>>>
>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>>> wrote:
>>>
>>>      Arun,
>>>
>>>      I've given you permission to edit the wiki. Let me know if
>>> you run
>>>
>>> into any
>>>
>>>      issues.
>>>
>>>      -Ewen
>>>
>>>      On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co
>>> rp.jp> <am...@yahoo-corp.jp>
>>>
>>> wrote:
>>>
>>>
>>>      > Thanks Michal. I don\u2019t have the access yet [arunmathew88].
>>> Should I
>>>
>>> be
>>>
>>>      > sending a separate mail for this?
>>>      >
>>>      > I thought one of the person following this thread would be
>>> able to
>>>
>>> give me
>>>
>>>      > access.
>>>      >
>>>      >
>>>      >
>>>      > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>>      > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>      > *Date: *Friday, April 7, 2017 at 17:16
>>>      > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>>      > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>> semantics
>>>      >
>>>      >
>>>      >
>>>      > Hi Arun,
>>>      >
>>>      > I was thinking along the same lines as you, listing the use
>>> cases
>>>
>>> on the
>>>
>>>      > wiki, but didn't find time to get around doing that yet.
>>>      > Don't mind if you do it if you have access now.
>>>      > I was thinking it would be nice if, once we have the use
>>> cases
>>>
>>> listed,
>>>
>>>      > people could use likes to up-vote the use cases similar to
>>> what
>>>
>>> they're
>>>
>>>      > working on.
>>>      >
>>>      > I should have a bit more time to action this in the next
>>> few days,
>>>
>>> but
>>>
>>>      > happy for you to do it if you can beat me to it ;-)
>>>      >
>>>      > Cheers,
>>>      > Michal
>>>      >
>>>      > On 07/04/17 04:39, Arun Mathew wrote:
>>>      >
>>>      > Sure, Thanks Matthias. My id is [arunmathew88].
>>>      >
>>>      >
>>>      >
>>>      > Of course. I was thinking of a subpage where people can
>>>
>>> collaborate.
>>>
>>>      >
>>>      >
>>>      >
>>>      > Will do as per Michael\u2019s suggestion.
>>>      >
>>>      >
>>>      >
>>>      > Regards,
>>>      >
>>>      > Arun Mathew
>>>      >
>>>      >
>>>      >
>>>      > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>>> <
>>>
>>> matthias@confluent.io> wrote:
>>>
>>>      >
>>>      >
>>>      >
>>>      >     Please share your Wiki-ID and a committer can give you
>>> write
>>>
>>> access.
>>>
>>>      >
>>>      >
>>>      >
>>>      >     Btw: as you did not initiate the KIP, you should not
>>> change the
>>>
>>> KIP
>>>
>>>      >
>>>      >     without the permission of the original author -- in
>>> this case
>>>
>>> Michael.
>>>
>>>      >
>>>      >
>>>      >
>>>      >     So you might also just share your thought over the
>>> mailing list
>>>
>>> and
>>>
>>>      >
>>>      >     Michael can update the KIP page. Or, as an alternative,
>>> just
>>>
>>> create a
>>>
>>>      >
>>>      >     subpage for the KIP page.
>>>      >
>>>      >
>>>      >
>>>      >     @Michael: WDYT?
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >     -Matthias
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>      >
>>>      >     > Hi Jay,
>>>      >
>>>      >     >           Thanks for the advise, I would like to list
>>> down
>>>
>>> the use cases as
>>>
>>>      >
>>>      >     > per your suggestion. But it seems I don't have write
>>>
>>> permission to the
>>>
>>>      >
>>>      >     > Apache Kafka Confluent Space. Whom shall I request
>>> for it?
>>>      >
>>>      >     >
>>>      >
>>>      >     > Regarding your last question. We are using a patch in
>>> our
>>>
>>> production system
>>>
>>>      >
>>>      >     > which does exactly this.
>>>      >
>>>      >     > We window by the event time, but trigger punctuate in
>>>
>>> <punctuate interval>
>>>
>>>      >
>>>      >     > duration of system time, in the absence of an event
>>> crossing
>>>
>>> the punctuate
>>>
>>>      >
>>>      >     > event time.
>>>      >
>>>      >     >
>>>      >
>>>      >     > We are using Kafka Stream for our Audit Trail, where
>>> we need
>>>
>>> to output the
>>>
>>>      >
>>>      >     > event counts on each topic on each cluster aggregated
>>> over a
>>>
>>> 1 minute
>>>
>>>      >
>>>      >     > window. We have to use event time to be able to cross
>>> check
>>>
>>> the counts. But
>>>
>>>      >
>>>      >     > we need to trigger punctuate [aggregate event pushes]
>>> by
>>>
>>> system time in the
>>>
>>>      >
>>>      >     > absence of events. Otherwise the event counts for
>>> unexpired
>>>
>>> windows would
>>>
>>>      >
>>>      >     > be 0 which is bad.
>>>      >
>>>      >     >
>>>      >
>>>      >     > "Maybe a hybrid solution works: I window by event
>>> time but
>>>
>>> trigger results
>>>
>>>      >
>>>      >     > by system time for windows that have updated? Not
>>> really sure
>>>
>>> the details
>>>
>>>      >
>>>      >     > of making that work. Does that work? Are there
>>> concrete
>>>
>>> examples where you
>>>
>>>      >
>>>      >     > actually want the current behavior?"
>>>      >
>>>      >     >
>>>      >
>>>      >     > --
>>>      >
>>>      >     > With Regards,
>>>      >
>>>      >     >
>>>      >
>>>      >     > Arun Mathew
>>>      >
>>>      >     > Yahoo! JAPAN Corporation
>>>      >
>>>      >     >
>>>      >
>>>      >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>
>>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>>
>>>      >
>>>      >     >
>>>      >
>>>      >     >> Hi Jay,
>>>      >
>>>      >     >>
>>>      >
>>>      >     >> The hybrid solution is exactly what I expect and
>>> need for
>>>
>>> our use cases
>>>
>>>      >
>>>      >     >> when dealing with telecom data.
>>>      >
>>>      >     >>
>>>      >
>>>      >     >> Thanks
>>>      >
>>>      >     >> Tianji
>>>      >
>>>      >     >>
>>>      >
>>>      >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>
>>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>>
>>>      >
>>>      >     >>
>>>      >
>>>      >     >>> Hey guys,
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>> One thing I've always found super important for
>>> this kind
>>>
>>> of design work
>>>
>>>      >
>>>      >     >> is
>>>      >
>>>      >     >>> to do a really good job of cataloging the landscape
>>> of use
>>>
>>> cases and how
>>>
>>>      >
>>>      >     >>> prevalent each one is. By that I mean not just
>>> listing lots
>>>
>>> of uses, but
>>>
>>>      >
>>>      >     >>> also grouping them into categories that
>>> functionally need
>>>
>>> the same thing.
>>>
>>>      >
>>>      >     >>> In the absence of this it is very hard to reason
>>> about
>>>
>>> design proposals.
>>>
>>>      >
>>>      >     >>> From the proposals so far I think we have a lot of
>>>
>>> discussion around
>>>
>>>      >
>>>      >     >>> possible apis, but less around what the user needs
>>> for
>>>
>>> different use
>>>
>>>      >
>>>      >     >> cases
>>>      >
>>>      >     >>> and how they would implement that using the api.
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>> Here is an example:
>>>      >
>>>      >     >>> You aggregate click and impression data for a
>>> reddit like
>>>
>>> site. Every ten
>>>
>>>      >
>>>      >     >>> minutes you want to output a ranked list of the top
>>> 10
>>>
>>> articles ranked by
>>>
>>>      >
>>>      >     >>> clicks/impressions for each geographical area. I
>>> want to be
>>>
>>> able run this
>>>
>>>      >
>>>      >     >>> in steady state as well as rerun to regenerate
>>> results (or
>>>
>>> catch up if it
>>>
>>>      >
>>>      >     >>> crashes).
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>> There are a couple of tricky things that seem to
>>> make this
>>>
>>> hard with
>>>
>>>      >
>>>      >     >> either
>>>      >
>>>      >     >>> of the options proposed:
>>>      >
>>>      >     >>> 1. If I emit this data using event time I have the
>>> problem
>>>
>>> described
>>>
>>>      >
>>>      >     >> where
>>>      >
>>>      >     >>> a geographical region with no new clicks or
>>> impressions
>>>
>>> will fail to
>>>
>>>      >
>>>      >     >> output
>>>      >
>>>      >     >>> results.
>>>      >
>>>      >     >>> 2. If I emit this data using system time I have the
>>> problem
>>>
>>> that when
>>>
>>>      >
>>>      >     >>> reprocessing data my window may not be ten minutes
>>> but 10
>>>
>>> hours if my
>>>
>>>      >
>>>      >     >>> processing is very fast so it dramatically changes
>>> the
>>>
>>> output.
>>>
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>> Maybe a hybrid solution works: I window by event
>>> time but
>>>
>>> trigger results
>>>
>>>      >
>>>      >     >>> by system time for windows that have updated? Not
>>> really
>>>
>>> sure the details
>>>
>>>      >
>>>      >     >>> of making that work. Does that work? Are there
>>> concrete
>>>
>>> examples where
>>>
>>>      >
>>>      >     >> you
>>>      >
>>>      >     >>> actually want the current behavior?
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>> -Jay
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>
>>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>>
>>>      >
>>>      >     >>> wrote:
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>>> Hi All,
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> Thanks for the KIP. We were also in need of a
>>> mechanism to
>>>
>>> trigger
>>>
>>>      >
>>>      >     >>>> punctuate in the absence of events.
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> As I described in [
>>>      >
>>>      >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>      >
>>>      >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>      >
>>>      >     >>>> plugin.system.issuetabpanels:comment-
>>> tabpanel#comment-
>>>
>>> 15926036
>>>
>>>      >
>>>      >     >>>> ],
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>>    - Our approached involved using the event time
>>> by
>>>
>>> default.
>>>
>>>      >
>>>      >     >>>>    - The method to check if there is any punctuate
>>> ready
>>>
>>> in the
>>>
>>>      >
>>>      >     >>>>    PunctuationQueue is triggered via the any event
>>>
>>> received by the
>>>
>>>      >
>>>      >     >> stream
>>>      >
>>>      >     >>>>    tread, or at the polling intervals in the
>>> absence of
>>>
>>> any events.
>>>
>>>      >
>>>      >     >>>>    - When we create Punctuate objects (which
>>> contains the
>>>
>>> next event
>>>
>>>      >
>>>      >     >> time
>>>      >
>>>      >     >>>>    for punctuation and interval), we also record
>>> the
>>>
>>> creation time
>>>
>>>      >
>>>      >     >>> (system
>>>      >
>>>      >     >>>>    time).
>>>      >
>>>      >     >>>>    - While checking for maturity of Punctuate
>>> Schedule by
>>>      >
>>>      >     >> mayBePunctuate
>>>      >
>>>      >     >>>>    method, we also check if the system clock has
>>> elapsed
>>>
>>> the punctuate
>>>
>>>      >
>>>      >     >>>>    interval since the schedule creation time.
>>>      >
>>>      >     >>>>    - In the absence of any event, or in the
>>> absence of any
>>>
>>> event for
>>>
>>>      >
>>>      >     >> one
>>>      >
>>>      >     >>>>    topic in the partition group assigned to the
>>> stream
>>>
>>> task, the system
>>>
>>>      >
>>>      >     >>>> time
>>>      >
>>>      >     >>>>    will elapse the interval and we trigger a
>>> punctuate
>>>
>>> using the
>>>
>>>      >
>>>      >     >> expected
>>>      >
>>>      >     >>>>    punctuation event time.
>>>      >
>>>      >     >>>>    - we then create the next punctuation schedule
>>> as
>>>
>>> punctuation event
>>>
>>>      >
>>>      >     >>> time
>>>      >
>>>      >     >>>>    + punctuation interval, [again recording the
>>> system
>>>
>>> time of creation
>>>
>>>      >
>>>      >     >>> of
>>>      >
>>>      >     >>>> the
>>>      >
>>>      >     >>>>    schedule].
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> We call this a Hybrid Punctuate. Of course, this
>>> approach
>>>
>>> has pros and
>>>
>>>      >
>>>      >     >>>> cons.
>>>      >
>>>      >     >>>> Pros
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>>    - Punctuates will happen in <punctuate
>>> interval> time
>>>
>>> duration at
>>>
>>>      >
>>>      >     >> max
>>>      >
>>>      >     >>> in
>>>      >
>>>      >     >>>>    terms of system time.
>>>      >
>>>      >     >>>>    - The semantics as a whole continues to revolve
>>> around
>>>
>>> event time.
>>>
>>>      >
>>>      >     >>>>    - We can use the old data [old timestamps] to
>>> rerun any
>>>
>>> experiments
>>>
>>>      >
>>>      >     >> or
>>>      >
>>>      >     >>>>    tests.
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> Cons
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>>    - In case the  <punctuate interval> is not a
>>> time
>>>
>>> duration [say
>>>
>>>      >
>>>      >     >>> logical
>>>      >
>>>      >     >>>>    time/event count], then the approach might not
>>> be
>>>
>>> meaningful.
>>>
>>>      >
>>>      >     >>>>    - In case there is a case where we have to wait
>>> for an
>>>
>>> actual event
>>>
>>>      >
>>>      >     >>> from
>>>      >
>>>      >     >>>>    a low event rate partition in the partition
>>> group, this
>>>
>>> approach
>>>
>>>      >
>>>      >     >> will
>>>      >
>>>      >     >>>> jump
>>>      >
>>>      >     >>>>    the gun.
>>>      >
>>>      >     >>>>    - in case the event processing cannot catch up
>>> with the
>>>
>>> event rate
>>>
>>>      >
>>>      >     >> and
>>>      >
>>>      >     >>>>    the expected timestamp events gets queued for
>>> long
>>>
>>> time, this
>>>
>>>      >
>>>      >     >> approach
>>>      >
>>>      >     >>>>    might jump the gun.
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> I believe the above approach and discussion goes
>>> close to
>>>
>>> the approach
>>>
>>>      >
>>>      >     >> A.
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> -----------
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> I like the idea of having an even count based
>>> punctuate.
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> -----------
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> I agree with the discussion around approach C,
>>> that we
>>>
>>> should provide
>>>
>>>      >
>>>      >     >> the
>>>      >
>>>      >     >>>> user with the option to choose system time or
>>> event time
>>>
>>> based
>>>
>>>      >
>>>      >     >>> punctuates.
>>>      >
>>>      >     >>>> But I believe that the user predominantly wants to
>>> use
>>>
>>> event time while
>>>
>>>      >
>>>      >     >>> not
>>>      >
>>>      >     >>>> missing out on regular punctuates due to event
>>> delays or
>>>
>>> event
>>>
>>>      >
>>>      >     >> absences.
>>>      >
>>>      >     >>>> Hence a complex punctuate option as Matthias
>>> mentioned
>>>
>>> (quoted below)
>>>
>>>      >
>>>      >     >>> would
>>>      >
>>>      >     >>>> be most apt.
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> "- We might want to add "complex" schedules later
>>> on
>>>
>>> (like, punctuate
>>>
>>>      >
>>>      >     >> on
>>>      >
>>>      >     >>>> every 10 seconds event-time or 60 seconds system-
>>> time
>>>
>>> whatever comes
>>>
>>>      >
>>>      >     >>>> first)."
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> -----------
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> I think I read somewhere that Kafka Streams
>>> started with
>>>
>>> System Time as
>>>
>>>      >
>>>      >     >>> the
>>>      >
>>>      >     >>>> punctuation standard, but was later changed to
>>> Event Time.
>>>
>>> I guess
>>>
>>>      >
>>>      >     >> there
>>>      >
>>>      >     >>>> would be some good reason behind it. As Kafka
>>> Streams want
>>>
>>> to evolve
>>>
>>>      >
>>>      >     >> more
>>>      >
>>>      >     >>>> on the Stream Processing front, I believe the
>>> emphasis on
>>>
>>> event time
>>>
>>>      >
>>>      >     >>> would
>>>      >
>>>      >     >>>> remain quite strong.
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> With Regards,
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> Arun Mathew
>>>      >
>>>      >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>
>>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>>
>>>      >
>>>      >     >> wrote:
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>>> Yeah I like PuncutationType much better; I just
>>> threw
>>>
>>> Time out there
>>>
>>>      >
>>>      >     >>>>> more as a strawman than an actual suggestion ;) I
>>> still
>>>
>>> think it's
>>>
>>>      >
>>>      >     >>>>> worth considering what this buys us over an
>>> additional
>>>
>>> callback. I
>>>
>>>      >
>>>      >     >>>>> foresee a number of punctuate implementations
>>> following
>>>
>>> this pattern:
>>>
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>> public void punctuate(PunctuationType type) {
>>>      >
>>>      >     >>>>>     switch (type) {
>>>      >
>>>      >     >>>>>         case EVENT_TIME:
>>>      >
>>>      >     >>>>>             methodA();
>>>      >
>>>      >     >>>>>             break;
>>>      >
>>>      >     >>>>>         case SYSTEM_TIME:
>>>      >
>>>      >     >>>>>             methodB();
>>>      >
>>>      >     >>>>>             break;
>>>      >
>>>      >     >>>>>     }
>>>      >
>>>      >     >>>>> }
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>> I guess one advantage of this approach is we
>>> could add
>>>
>>> additional
>>>
>>>      >
>>>      >     >>>>> punctuation types later in a backwards compatible
>>> way
>>>
>>> (like event
>>>
>>>      >
>>>      >     >> count
>>>      >
>>>      >     >>>>> as you mentioned).
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>> -Tommy
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>> Sax wrote:
>>>      >
>>>      >     >>>>>> That sounds promising.
>>>      >
>>>      >     >>>>>>
>>>      >
>>>      >     >>>>>> I am just wondering if `Time` is the best name.
>>> Maybe we
>>>
>>> want to
>>>
>>>      >
>>>      >     >> add
>>>      >
>>>      >     >>>>>> other non-time based punctuations at some point
>>> later. I
>>>
>>> would
>>>
>>>      >
>>>      >     >>>>>> suggest
>>>      >
>>>      >     >>>>>>
>>>      >
>>>      >     >>>>>> enum PunctuationType {
>>>      >
>>>      >     >>>>>>   EVENT_TIME,
>>>      >
>>>      >     >>>>>>   SYSTEM_TIME,
>>>      >
>>>      >     >>>>>> }
>>>      >
>>>      >     >>>>>>
>>>      >
>>>      >     >>>>>> or similar. Just to keep the door open -- it's
>>> easier to
>>>
>>> add new
>>>
>>>      >
>>>      >     >>>>>> stuff
>>>      >
>>>      >     >>>>>> if the name is more generic.
>>>      >
>>>      >     >>>>>>
>>>      >
>>>      >     >>>>>>
>>>      >
>>>      >     >>>>>> -Matthias
>>>      >
>>>      >     >>>>>>
>>>      >
>>>      >     >>>>>>
>>>      >
>>>      >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>> I agree that the framework providing and
>>> managing the
>>>
>>> notion of
>>>
>>>      >
>>>      >     >>>>>>> stream
>>>      >
>>>      >     >>>>>>> time is valuable and not something we would
>>> want to
>>>
>>> delegate to
>>>
>>>      >
>>>      >     >> the
>>>      >
>>>      >     >>>>>>> tasks. I'm not entirely convinced that a
>>> separate
>>>
>>> callback
>>>
>>>      >
>>>      >     >> (option
>>>      >
>>>      >     >>>>>>> C)
>>>      >
>>>      >     >>>>>>> is that messy (it could just be a default
>>> method with
>>>
>>> an empty
>>>
>>>      >
>>>      >     >>>>>>> implementation), but if we wanted a single API
>>> to
>>>
>>> handle both
>>>
>>>      >
>>>      >     >>>>>>> cases,
>>>      >
>>>      >     >>>>>>> how about something like the following?
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>> enum Time {
>>>      >
>>>      >     >>>>>>>    STREAM,
>>>      >
>>>      >     >>>>>>>    CLOCK
>>>      >
>>>      >     >>>>>>> }
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>> Then on ProcessorContext:
>>>      >
>>>      >     >>>>>>> context.schedule(Time time, long interval)  //
>>> We could
>>>
>>> allow
>>>
>>>      >
>>>      >     >> this
>>>      >
>>>      >     >>>>>>> to
>>>      >
>>>      >     >>>>>>> be called once for each value of time to mix
>>>
>>> approaches.
>>>
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>> Then the Processor API becomes:
>>>      >
>>>      >     >>>>>>> punctuate(Time time) // time here denotes which
>>>
>>> schedule resulted
>>>
>>>      >
>>>      >     >>>>>>> in
>>>      >
>>>      >     >>>>>>> this call.
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>> Thoughts?
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>> Sax
>>>
>>> wrote:
>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> I was thinking about the four options you
>>> proposed in
>>>
>>> more
>>>
>>>      >
>>>      >     >>>>>>>> details
>>>      >
>>>      >     >>>>>>>> and
>>>      >
>>>      >     >>>>>>>> this are my thoughts:
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> (A) You argue, that users can still
>>> "punctuate" on
>>>
>>> event-time
>>>
>>>      >
>>>      >     >> via
>>>      >
>>>      >     >>>>>>>> process(), but I am not sure if this is
>>> possible.
>>>
>>> Note, that
>>>
>>>      >
>>>      >     >>>>>>>> users
>>>      >
>>>      >     >>>>>>>> only
>>>      >
>>>      >     >>>>>>>> get record timestamps via context.timestamp().
>>> Thus,
>>>
>>> users
>>>
>>>      >
>>>      >     >> would
>>>      >
>>>      >     >>>>>>>> need
>>>      >
>>>      >     >>>>>>>> to
>>>      >
>>>      >     >>>>>>>> track the time progress per partition (based
>>> on the
>>>
>>> partitions
>>>
>>>      >
>>>      >     >>>>>>>> they
>>>      >
>>>      >     >>>>>>>> obverse via context.partition(). (This alone
>>> puts a
>>>
>>> huge burden
>>>
>>>      >
>>>      >     >>>>>>>> on
>>>      >
>>>      >     >>>>>>>> the
>>>      >
>>>      >     >>>>>>>> user by itself.) However, users are not
>>> notified at
>>>
>>> startup
>>>
>>>      >
>>>      >     >> what
>>>      >
>>>      >     >>>>>>>> partitions are assigned, and user are not
>>> notified
>>>
>>> when
>>>
>>>      >
>>>      >     >>>>>>>> partitions
>>>      >
>>>      >     >>>>>>>> get
>>>      >
>>>      >     >>>>>>>> revoked. Because this information is not
>>> available,
>>>
>>> it's not
>>>
>>>      >
>>>      >     >>>>>>>> possible
>>>      >
>>>      >     >>>>>>>> to
>>>      >
>>>      >     >>>>>>>> "manually advance" stream-time, and thus
>>> event-time
>>>
>>> punctuation
>>>
>>>      >
>>>      >     >>>>>>>> within
>>>      >
>>>      >     >>>>>>>> process() seems not to be possible -- or do
>>> you see a
>>>
>>> way to
>>>
>>>      >
>>>      >     >> get
>>>      >
>>>      >     >>>>>>>> it
>>>      >
>>>      >     >>>>>>>> done? And even if, it might still be too
>>> clumsy to
>>>
>>> use.
>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> (B) This does not allow to mix both
>>> approaches, thus
>>>
>>> limiting
>>>
>>>      >
>>>      >     >>>>>>>> what
>>>      >
>>>      >     >>>>>>>> users
>>>      >
>>>      >     >>>>>>>> can do.
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> (C) This should give all flexibility we need.
>>> However,
>>>
>>> just
>>>
>>>      >
>>>      >     >>>>>>>> adding
>>>      >
>>>      >     >>>>>>>> one
>>>      >
>>>      >     >>>>>>>> more method seems to be a solution that is too
>>> simple
>>>
>>> (cf my
>>>
>>>      >
>>>      >     >>>>>>>> comments
>>>      >
>>>      >     >>>>>>>> below).
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>> sure how
>>>
>>> a user
>>>
>>>      >
>>>      >     >>>>>>>> could
>>>      >
>>>      >     >>>>>>>> enable system-time and event-time punctuation
>>> in
>>>
>>> parallel.
>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> Overall options (C) seems to be the most
>>> promising
>>>
>>> approach to
>>>
>>>      >
>>>      >     >>>>>>>> me.
>>>      >
>>>      >     >>>>>>>> Because I also favor a clean API, we might
>>> keep
>>>
>>> current
>>>
>>>      >
>>>      >     >>>>>>>> punctuate()
>>>      >
>>>      >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>> at some
>>>
>>> later
>>>
>>>      >
>>>      >     >>>>>>>> point
>>>      >
>>>      >     >>>>>>>> when
>>>      >
>>>      >     >>>>>>>> people use the "new punctuate API".
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> Couple of follow up questions:
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> - I am wondering, if we should have two
>>> callback
>>>
>>> methods or
>>>
>>>      >
>>>      >     >> just
>>>      >
>>>      >     >>>>>>>> one
>>>      >
>>>      >     >>>>>>>> (ie, a unified for system and event time
>>> punctuation
>>>
>>> or one for
>>>
>>>      >
>>>      >     >>>>>>>> each?).
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> - If we have one, how can the user figure out,
>>> which
>>>
>>> condition
>>>
>>>      >
>>>      >     >>>>>>>> did
>>>      >
>>>      >     >>>>>>>> trigger?
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> - How would the API look like, for registering
>>>
>>> different
>>>
>>>      >
>>>      >     >>>>>>>> punctuate
>>>      >
>>>      >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> - We might want to add "complex" schedules
>>> later on
>>>
>>> (like,
>>>
>>>      >
>>>      >     >>>>>>>> punctuate
>>>      >
>>>      >     >>>>>>>> on
>>>      >
>>>      >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>> system-time
>>>
>>> whatever
>>>
>>>      >
>>>      >     >>>>>>>> comes
>>>      >
>>>      >     >>>>>>>> first). I don't say we should add this right
>>> away, but
>>>
>>> we might
>>>
>>>      >
>>>      >     >>>>>>>> want
>>>      >
>>>      >     >>>>>>>> to
>>>      >
>>>      >     >>>>>>>> define the API in a way, that it allows
>>> extensions
>>>
>>> like this
>>>
>>>      >
>>>      >     >>>>>>>> later
>>>      >
>>>      >     >>>>>>>> on,
>>>      >
>>>      >     >>>>>>>> without redesigning the API (ie, the API
>>> should be
>>>
>>> designed
>>>
>>>      >
>>>      >     >>>>>>>> extensible)
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> - Did you ever consider count-based
>>> punctuation?
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> I understand, that you would like to solve a
>>> simple
>>>
>>> problem,
>>>
>>>      >
>>>      >     >> but
>>>      >
>>>      >     >>>>>>>> we
>>>      >
>>>      >     >>>>>>>> learned from the past, that just "adding some
>>> API"
>>>
>>> quickly
>>>
>>>      >
>>>      >     >> leads
>>>      >
>>>      >     >>>>>>>> to a
>>>      >
>>>      >     >>>>>>>> not very well defined API that needs time
>>> consuming
>>>
>>> clean up
>>>
>>>      >
>>>      >     >>>>>>>> later on
>>>      >
>>>      >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>> holistic
>>>      >
>>>      >     >>>>>>>> punctuation
>>>      >
>>>      >     >>>>>>>> KIP
>>>      >
>>>      >     >>>>>>>> with this from the beginning on to avoid later
>>> painful
>>>      >
>>>      >     >> redesign.
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> -Matthias
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>>
>>>      >
>>>      >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>> Thanks Thomas,
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>> I'm also wary of changing the existing
>>> semantics of
>>>      >
>>>      >     >> punctuate,
>>>      >
>>>      >     >>>>>>>>> for
>>>      >
>>>      >     >>>>>>>>> backward compatibility reasons, although I
>>> like the
>>>      >
>>>      >     >> conceptual
>>>      >
>>>      >     >>>>>>>>> simplicity of that option.
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>> Adding a new method to me feels safer but, in
>>> a way,
>>>
>>> uglier.
>>>
>>>      >
>>>      >     >> I
>>>      >
>>>      >     >>>>>>>>> added
>>>      >
>>>      >     >>>>>>>>> this to the KIP now as option (C).
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>> more
>>>
>>> flexible,
>>>
>>>      >
>>>      >     >> as
>>>      >
>>>      >     >>>>>>>>> it
>>>      >
>>>      >     >>>>>>>>> allows
>>>      >
>>>      >     >>>>>>>>> you to return any value, you're not limited
>>> to event
>>>
>>> time or
>>>
>>>      >
>>>      >     >>>>>>>>> system
>>>      >
>>>      >     >>>>>>>>> time
>>>      >
>>>      >     >>>>>>>>> (although I don't see an actual use case
>>> where you
>>>
>>> might need
>>>
>>>      >
>>>      >     >>>>>>>>> anything
>>>      >
>>>      >     >>>>>>>>> else then those two). Hence I also proposed
>>> the
>>>
>>> option to
>>>
>>>      >
>>>      >     >> allow
>>>      >
>>>      >     >>>>>>>>> users
>>>      >
>>>      >     >>>>>>>>> to, effectively, decide what "stream time" is
>>> for
>>>
>>> them given
>>>
>>>      >
>>>      >     >>>>>>>>> the
>>>      >
>>>      >     >>>>>>>>> presence or absence of messages, much like
>>> they can
>>>
>>> decide
>>>
>>>      >
>>>      >     >> what
>>>      >
>>>      >     >>>>>>>>> msg
>>>      >
>>>      >     >>>>>>>>> time
>>>      >
>>>      >     >>>>>>>>> means for them using the TimestampExtractor.
>>> What do
>>>
>>> you
>>>
>>>      >
>>>      >     >> think
>>>      >
>>>      >     >>>>>>>>> about
>>>      >
>>>      >     >>>>>>>>> that? This is probably most flexible but also
>>> most
>>>      >
>>>      >     >> complicated.
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>> All comments appreciated.
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>> Cheers,
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>> Michal
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>
>>>      >
>>>      >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>      >
>>>      >     >>>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>> Although I fully agree we need a way to
>>> trigger
>>>
>>> periodic
>>>
>>>      >
>>>      >     >>>>>>>>>> processing
>>>      >
>>>      >     >>>>>>>>>> that is independent from whether and when
>>> messages
>>>
>>> arrive,
>>>
>>>      >
>>>      >     >>>>>>>>>> I'm
>>>      >
>>>      >     >>>>>>>>>> not sure
>>>      >
>>>      >     >>>>>>>>>> I like the idea of changing the existing
>>> semantics
>>>
>>> across
>>>
>>>      >
>>>      >     >> the
>>>      >
>>>      >     >>>>>>>>>> board.
>>>      >
>>>      >     >>>>>>>>>> What if we added an additional callback to
>>> Processor
>>>
>>> that
>>>
>>>      >
>>>      >     >> can
>>>      >
>>>      >     >>>>>>>>>> be
>>>      >
>>>      >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>> always
>>>
>>> called at
>>>
>>>      >
>>>      >     >>>>>>>>>> fixed, wall
>>>      >
>>>      >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>> have to
>>>
>>> give
>>>
>>>      >
>>>      >     >> up
>>>      >
>>>      >     >>>>>>>>>> the
>>>      >
>>>      >     >>>>>>>>>> notion
>>>      >
>>>      >     >>>>>>>>>> of stream time to be able to do periodic
>>> processing.
>>>      >
>>>      >     >>>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>> Borowiecki
>>>
>>> wrote:
>>>
>>>      >
>>>      >     >>>>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>>> Hi all,
>>>      >
>>>      >     >>>>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>
>>> punctuate
>>>
>>>      >
>>>      >     >>>>>>>>>>> semantics
>>>      >
>>>      >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>
>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
>>> confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>
>>>      >
>>>      >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI
>>> P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>
>>> 138%
>>>
>>>      >
>>>      >     >>>>>>>>>>> 3A+C
>>>      >
>>>      >     >>>>>>>>>>> hange+
>>>      >
>>>      >     >>>>>>>>>>> punctuate+semantics>
>>>      >
>>>      >     >>>>>>>>>>> .
>>>      >
>>>      >     >>>>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>>> Appreciating there can be different views
>>> on
>>>
>>> system-time
>>>
>>>      >
>>>      >     >> vs
>>>      >
>>>      >     >>>>>>>>>>> event-
>>>      >
>>>      >     >>>>>>>>>>> time
>>>      >
>>>      >     >>>>>>>>>>> semantics for punctuation depending on use-
>>> case and
>>>
>>> the
>>>
>>>      >
>>>      >     >>>>>>>>>>> importance of
>>>      >
>>>      >     >>>>>>>>>>> backwards compatibility of any such change,
>>> I've
>>>
>>> left it
>>>
>>>      >
>>>      >     >>>>>>>>>>> quite
>>>      >
>>>      >     >>>>>>>>>>> open
>>>      >
>>>      >     >>>>>>>>>>> and
>>>      >
>>>      >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>
>>> progresses.
>>>
>>>      >
>>>      >     >>>>>>>>>>>
>>>      >
>>>      >     >>>>>>>>>>> Thanks,
>>>      >
>>>      >     >>>>>>>>>>> Michal
>>>      >
>>>      >     >>>>>>> --
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>>     Tommy Becker
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>>     Senior Software Engineer
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>>     tivo.com
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>> ________________________________
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>>>> This email and any attachments may contain
>>> confidential
>>>
>>> and
>>>
>>>      >
>>>      >     >>>>>>> privileged material for the sole use of the
>>> intended
>>>
>>> recipient.
>>>
>>>      >
>>>      >     >> Any
>>>      >
>>>      >     >>>>>>> review, copying, or distribution of this email
>>> (or any
>>>      >
>>>      >     >> attachments)
>>>      >
>>>      >     >>>>>>> by others is prohibited. If you are not the
>>> intended
>>>
>>> recipient,
>>>
>>>      >
>>>      >     >>>>>>> please contact the sender immediately and
>>> permanently
>>>
>>> delete this
>>>
>>>      >
>>>      >     >>>>>>> email and any attachments. No employee or agent
>>> of TiVo
>>>
>>> Inc. is
>>>
>>>      >
>>>      >     >>>>>>> authorized to conclude any binding agreement on
>>> behalf
>>>
>>> of TiVo
>>>
>>>      >
>>>      >     >> Inc.
>>>      >
>>>      >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>> only be
>>>
>>> made by a
>>>
>>>      >
>>>      >     >>>>>>> signed written agreement.
>>>      >
>>>      >     >>>>>>>
>>>      >
>>>      >     >>>>> --
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>>     Tommy Becker
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>>     Senior Software Engineer
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>>     tivo.com
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>> ________________________________
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>> This email and any attachments may contain
>>> confidential
>>>
>>> and
>>>
>>>      >
>>>      >     >> privileged
>>>      >
>>>      >     >>>>> material for the sole use of the intended
>>> recipient. Any
>>>
>>> review,
>>>
>>>      >
>>>      >     >>> copying,
>>>      >
>>>      >     >>>>> or distribution of this email (or any
>>> attachments) by
>>>
>>> others is
>>>
>>>      >
>>>      >     >>>> prohibited.
>>>      >
>>>      >     >>>>> If you are not the intended recipient, please
>>> contact the
>>>
>>> sender
>>>
>>>      >
>>>      >     >>>>> immediately and permanently delete this email and
>>> any
>>>
>>> attachments. No
>>>
>>>      >
>>>      >     >>>>> employee or agent of TiVo Inc. is authorized to
>>> conclude
>>>
>>> any binding
>>>
>>>      >
>>>      >     >>>>> agreement on behalf of TiVo Inc. by email.
>>> Binding
>>>
>>> agreements with
>>>
>>>      >
>>>      >     >> TiVo
>>>      >
>>>      >     >>>>> Inc. may only be made by a signed written
>>> agreement.
>>>      >
>>>      >     >>>>>
>>>      >
>>>      >     >>>>
>>>      >
>>>      >     >>>
>>>      >
>>>      >     >>
>>>      >
>>>      >     >
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >
>>>      >
>>>      > --
>>>      >
>>>      > <http://www.openbet.com/> <http://www.openbet.com/>
>>>
>>>      >
>>>      > *Michal Borowiecki*
>>>      >
>>>      > *Senior Software Engineer L4*
>>>      >
>>>      > *T: *
>>>      >
>>>      > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>>      >
>>>      > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>>      >
>>>      >
>>>      >
>>>      > *E: *
>>>      >
>>>      > michal.borowiecki@openbet.com
>>>      >
>>>      > *W: *
>>>      >
>>>      > www.openbet.com
>>>      >
>>>      > *OpenBet Ltd*
>>>      >
>>>      > Chiswick Park Building 9
>>>      >
>>>      > 566 Chiswick High Rd
>>>      >
>>>      > London
>>>      >
>>>      > W4 5XT
>>>      >
>>>      > UK
>>>      >
>>>      > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>>      >
>>>      >
>>>      >
>>>      > This message is confidential and intended only for the
>>> addressee.
>>>
>>> If you
>>>
>>>      > have received this message in error, please immediately
>>> notify the
>>>      > postmaster@openbet.com and delete it from your system as
>>> well as
>>>
>>> any
>>>
>>>      > copies. The content of e-mails as well as traffic data may
>>> be
>>>
>>> monitored by
>>>
>>>      > OpenBet for employment and security purposes. To protect
>>> the
>>>
>>> environment
>>>
>>>      > please do not print this e-mail unless necessary. OpenBet
>>> Ltd.
>>>
>>> Registered
>>>
>>>      > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>> London,
>>>
>>> W4 5XT,
>>>
>>>      > United Kingdom. A company registered in England and Wales.
>>>
>>> Registered no.
>>>
>>>      > 3134634. VAT no. GB927523612
>>>      >
>>>      >
>>>      >
>>>
>>>
>>>
>>> --
>>>
>>>
>>>      Tommy Becker
>>>
>>>      Senior Software Engineer
>>>
>>>      O +1 919.460.4747 <%28919%29%20460-4747>
>>>
>>>
>>>      tivo.com
>>>
>>>
>>>
>>> ________________________________
>>>
>>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>>
>>>
>>> --
>>> <http://www.openbet.com/> Michal Borowiecki
>>> Senior Software Engineer L4
>>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>>
>>>
>>>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Damian Guy <da...@gmail.com>.
Thanks Michal,
I agree Transformer.punctuate should also be void, but we can deprecate
that too in favor of the new interface.

Thanks for the javadoc PR!

Cheers,
Damian

On Fri, 21 Apr 2017 at 09:31 Michal Borowiecki <
michal.borowiecki@openbet.com> wrote:

> Yes, that looks better to me.
>
> Note that punctuate on Transformer is currently returning a record, but I
> think it's ok to have all output records be sent via
> ProcessorContext.forward, which has to be used anyway if you want to send
> multiple records from one invocation of punctuate.
>
> This way it's consistent between Processor and Transformer.
>
>
> BTW, looking at this I found a glitch in the javadoc and put a comment
> there:
>
> https://github.com/apache/kafka/pull/2413/files#r112634612
>
> and PR: https://github.com/apache/kafka/pull/2884
>
> Cheers,
>
> Michal
> On 20/04/17 18:55, Damian Guy wrote:
>
> Hi Michal,
>
> Thanks for the KIP. I'd like to propose a bit more of a radical change to
> the API.
> 1. deprecate the punctuate method on Processor
> 2. create a new Functional Interface just for Punctuation, something like:
> interface Punctuator {
>     void punctuate(long timestamp)
> }
> 3. add a new schedule function to ProcessorContext: schedule(long
> interval, PunctuationType type, Punctuator callback)
> 4. deprecate the existing schedule function
>
> Thoughts?
>
> Thanks,
> Damian
>
> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
> michal.borowiecki@openbet.com> wrote:
>
>> Hi Thomas,
>>
>> I would say our use cases fall in the same category as yours.
>>
>> 1) One is expiry of old records, it's virtually identical to yours.
>>
>> 2) Second one is somewhat more convoluted but boils down to the same type
>> of design:
>>
>> Incoming messages carry a number of fields, including a timestamp.
>>
>> Outgoing messages contain derived fields, one of them (X) is depended on
>> by the timestamp input field (Y) and some other input field (Z).
>>
>> Since the output field X is derived in some non-trivial way, we don't
>> want to force the logic onto downstream apps. Instead we want to calculate
>> it in the Kafka Streams app, which means we re-calculate X as soon as the
>> timestamp in Y is reached (wall clock time) and send a message if it
>> changed (I say "if" because the derived field (X) is also conditional on
>> another input field Z).
>>
>> So we have kv stores with the records and an additional kv store with
>> timestamp->id mapping which act like an index where we periodically do a
>> ranged query.
>>
>> Initially we naively tried doing it in punctuate which of course didn't
>> work when there were no regular msgs on the input topic.
>> Since this was before 0.10.1 and state stores weren't query-able from
>> outside we created a "ticker" that produced msgs once per second onto
>> another topic and fed it into the same topology to trigger punctuate.
>> This didn't work either, which was much more surprising to us at the
>> time, because it was not obvious at all that punctuate is only triggered if
>> *all* input partitions receive messages regularly.
>> In the end we had to break this into 2 separate Kafka Streams. Main
>> transformer doesn't use punctuate but sends values of timestamp field Y and
>> the id to a "scheduler" topic where also the periodic ticks are sent. This
>> is consumed by the second topology and is its only input topic. There's a
>> transformer on that topic which populates and updates the time-based
>> indexes and polls them from punctuate. If the time in the timestamp
>> elapsed, the record id is sent to the main transformer, which
>> updates/deletes the record from the main kv store and forwards the
>> transformed record to the output topic.
>>
>> To me this setup feels horrendously complicated for what it does.
>>
>> We could incrementally improve on this since 0.10.1 to poll the
>> timestamp->id "index" stores from some code outside the KafkaStreams
>> topology so that at least we wouldn't need the extra topic for "ticks".
>> However, the ticks don't feel so hacky when you realise they give you
>> some hypothetical benefits in predictability. You can reprocess the
>> messages in a reproducible manner, since the topologies use event-time,
>> just that the event time is simply the wall-clock time fed into a topic by
>> the ticks. (NB in our use case we haven't yet found a need for this kind of
>> reprocessing).
>> To make that work though, we would have to have the stream time advance
>> based on the presence of msgs on the "tick" topic, regardless of the
>> presence of messages on the other input topic.
>>
>> Same as in the expiry use case, both the wall-clock triggered punctuate
>> and the hybrid would work to simplify this a lot.
>>
>> 3) Finally, I have a 3rd use case in the making but I'm still looking if
>> we can achieve it using session windows instead. I'll keep you posted if we
>> have to go with punctuate there too.
>>
>> Thanks,
>> Michal
>>
>>
>> On 11/04/17 20:52, Thomas Becker wrote:
>>
>> Here's an example that we currently have.  We have a streams processor
>> that does a transform from one topic into another. One of the fields in
>> the source topic record is an expiration time, and one of the functions
>> of the processor is to ensure that expired records get deleted promptly
>> after that time passes (typically days or weeks after the message was
>> originally produced). To do that, the processor keeps a state store of
>> keys and expiration times, iterates that store in punctuate(), and
>> emits delete (null) records for expired items. This needs to happen at
>> some minimum interval regardless of the incoming message rate of the
>> source topic.
>>
>> In this scenario, the expiration of records is the primary function of
>> punctuate, and therefore the key requirement is that the wall-clock
>> measured time between punctuate calls have some upper-bound. So a pure
>> wall-clock based schedule would be fine for our needs. But the proposed
>> "hybrid" system would also be acceptable if that satisfies a broader
>> range of use-cases.
>>
>> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>
>> I apologize for the longer email below.  To my defense, it started
>> out much
>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>> devil's
>> advocate for a number of arguments brought forth in order to help
>> improve
>> this KIP -- I am not implying I necessarily disagree with the
>> arguments.
>>
>> That aside, here are some further thoughts.
>>
>> First, there are (at least?) two categories for actions/behavior you
>> invoke
>> via punctuate():
>>
>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>> to
>> periodically commit to a custom store, to do metrics/logging).  Here,
>> the
>> impact of punctuate is typically not observable by other processing
>> nodes
>> in the topology.
>> 2. For controlling the emit frequency of downstream records.  Here,
>> the
>> punctuate is all about being observable by downstream processing
>> nodes.
>>
>> A few releases back, we introduced record caches (DSL) and state
>> store
>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>> relating to
>> (2) where some users needed to control -- here: limit -- the
>> downstream
>> output rate of Kafka Streams because the downstream systems/apps
>> would not
>> be able to keep up with the upstream output rate (Kafka scalability >
>> their
>> scalability).  The argument for KIP-63, which notably did not
>> introduce a
>> "trigger" API, was that such an interaction with downstream systems
>> is an
>> operational concern;  it should not impact the processing *logic* of
>> your
>> application, and thus we didn't want to complicate the Kafka Streams
>> API,
>> especially not the declarative DSL, with such operational concerns.
>>
>> This KIP's discussion on `punctuate()` takes us back in time (<--
>> sorry, I
>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>> observing that our conversation is moving more and more into the
>> direction
>> of explicit "triggers" because, so far, I have seen only motivations
>> for
>> use cases in category (2), but none yet for (1)?  For example, some
>> comments voiced here are about sth like "IF stream-time didn't
>> trigger
>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>> want
>> this, and if so, for which use cases and benefits?  Also, on a
>> related
>> note, whatever we are discussing here will impact state store caches
>> (Processor API) and perhaps also impact record caches (DSL), thus we
>> should
>> clarify any such impact here.
>>
>> Switching topics slightly.
>>
>> Jay wrote:
>>
>> One thing I've always found super important for this kind of design
>> work
>> is to do a really good job of cataloging the landscape of use cases
>> and
>> how prevalent each one is.
>>
>> +1 to this, as others have already said.
>>
>> Here, let me highlight -- just in case -- that when we talked about
>> windowing use cases in the recent emails, the Processor API (where
>> `punctuate` resides) does not have any notion of windowing at
>> all.  If you
>> want to do windowing *in the Processor API*, you must do so manually
>> in
>> combination with window stores.  For this reason I'd suggest to
>> discuss use
>> cases not just in general, but also in view of how you'd do so in the
>> Processor API vs. in the DSL.  Right now, changing/improving
>> `punctuate`
>> does not impact the DSL at all, unless we add new functionality to
>> it.
>>
>> Jay wrote in his strawman example:
>>
>> You aggregate click and impression data for a reddit like site.
>> Every ten
>> minutes you want to output a ranked list of the top 10 articles
>> ranked by
>> clicks/impressions for each geographical area. I want to be able
>> run this
>> in steady state as well as rerun to regenerate results (or catch up
>> if it
>> crashes).
>>
>> This is a good example for more than the obvious reason:  In KIP-63,
>> we
>> argued that the reason for saying "every ten minutes" above is not
>> necessarily about because you want to output data *exactly* after ten
>> minutes, but that you want to perform an aggregation based on 10-
>> minute
>> windows of input data; i.e., the point is about specifying the input
>> for
>> your aggregation, not or less about when the results of the
>> aggregation
>> should be send downstream.  To take an extreme example, you could
>> disable
>> record caches and let your app output a downstream update for every
>> incoming input record.  If the last input record was from at minute 7
>> of 10
>> (for a 10-min window), then what your app would output at minute 10
>> would
>> be identical to what it had already emitted at minute 7 earlier
>> anyways.
>> This is particularly true when we take late-arriving data into
>> account:  if
>> a late record arrived at minute 13, your app would (by default) send
>> a new
>> update downstream, even though the "original" 10 minutes have already
>> passed.
>>
>> Jay wrote...:
>>
>> There are a couple of tricky things that seem to make this hard
>> with
>>
>> either
>>
>> of the options proposed:
>> 1. If I emit this data using event time I have the problem
>> described where
>> a geographical region with no new clicks or impressions will fail
>> to
>>
>> output
>>
>> results.
>>
>> ...and Arun Mathew wrote:
>>
>>
>> We window by the event time, but trigger punctuate in <punctuate
>> interval>
>> duration of system time, in the absence of an event crossing the
>> punctuate
>> event time.
>>
>> So, given what I wrote above about the status quo and what you can
>> already
>> do with it, is the concern that the state store cache doesn't give
>> you
>> *direct* control over "forcing an output after no later than X
>> seconds [of
>> processing-time]" but only indirect control through a cache
>> size?  (Note
>> that I am not dismissing the claims why this might be helpful.)
>>
>> Arun Mathew wrote:
>>
>> We are using Kafka Stream for our Audit Trail, where we need to
>> output the
>> event counts on each topic on each cluster aggregated over a 1
>> minute
>> window. We have to use event time to be able to cross check the
>> counts.
>>
>> But
>>
>> we need to trigger punctuate [aggregate event pushes] by system
>> time in
>>
>> the
>>
>> absence of events. Otherwise the event counts for unexpired windows
>> would
>> be 0 which is bad.
>>
>> Isn't the latter -- "count would be 0" -- the problem between the
>> absence
>> of output vs. an output of 0, similar to the use of `Option[T]` in
>> Scala
>> and the difference between `None` and `Some(0)`?  That is, isn't the
>> root
>> cause that the downstream system interprets the absence of output in
>> a
>> particular way ("No output after 1 minute = I consider the output to
>> be
>> 0.")?  Arguably, you could also adapt the downstream system (if
>> possible)
>> to correctly handle the difference between absence of output vs.
>> output of
>> 0.  I am not implying that we shouldn't care about such a use case,
>> but
>> want to understand the motivation better. :-)
>>
>> Also, to add some perspective, in some related discussions we talked
>> about
>> how a Kafka Streams application should not worry or not be coupled
>> unnecessarily with such interpretation specifics in a downstream
>> system's
>> behavior.  After all, tomorrow your app's output might be consumed by
>> more
>> than just this one downstream system.  Arguably, Kafka Connect rather
>> than
>> Kafka Streams might be the best tool to link the universes of Kafka
>> and
>> downstream systems, including helping to reconcile the differences in
>> how
>> these systems interpret changes, updates, late-arriving data,
>> etc.  Kafka
>> Connect would allow you to decouple the Kafka Streams app's logical
>> processing from the specifics of downstream systems, thanks to
>> specific
>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>> this
>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>> but
>> it's currently awkward to use Connect for this", this might be a
>> problem we
>> can solve, too.)
>>
>> Switching topics slightly again.
>>
>> Thomas wrote:
>>
>> I'm not entirely convinced that a separate callback (option C)
>> is that messy (it could just be a default method with an empty
>> implementation), but if we wanted a single API to handle both
>> cases,
>> how about something like the following?
>>
>> enum Time {
>>    STREAM,
>>    CLOCK
>> }
>>
>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>> then
>> whatever the user is doing inside this method is a black box to Kafka
>> Streams (similar to how we have no idea what the user does inside a
>> `foreach` -- if the function passed to `foreach` writes to external
>> systems, then Kafka Streams is totally unaware of the fact).  We
>> won't
>> know, for example, if the stream-time action has a smaller "trigger"
>> frequency than the processing-time action.  Or, we won't know whether
>> the
>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>> minute of
>> stream-time or 1-minute of processing-time, whichever comes
>> first").  That
>> said, I am not certain yet whether we would need such knowledge
>> because,
>> when using the Processor API, most of the work and decisions must be
>> done
>> by the user anyways.  It would matter though if the concept of
>> "triggers"
>> were to bubble up into the DSL because in the DSL the management of
>> windowing, window stores, etc. must be done automatically by Kafka
>> Streams.
>>
>> [In any case, btw, we have the corner case where the user configured
>> the
>> stream-time to be processing-time (e.g. via wall-clock timestamp
>> extractor), at which point both punctuate variants are based on the
>> same
>> time semantics / timeline.]
>>
>> Again, I apologize for the wall of text.  Congratulations if you made
>> it
>> this far. :-)
>>
>> More than happy to hear your thoughts!
>> Michael
>>
>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>> wrote:
>>
>>
>> Thanks Matthias.
>> Sure, will correct it right away.
>>
>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>> wrote:
>>
>> Thanks for preparing this page!
>>
>> About terminology:
>>
>> You introduce the term "event time" -- but we should call this
>> "stream
>> time" -- "stream time" is whatever TimestampExtractor returns and
>> this
>> could be event time, ingestion time, or processing/wall-clock time.
>>
>> Does this make sense to you?
>>
>>
>>
>> -Matthias
>>
>>
>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>
>> Thanks Ewen.
>>
>> @Michal, @all, I have created a child page to start the Use Cases
>>
>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>
>>
>> @Tianji, Sorry for the delay. I am trying to make the patch
>> public.
>>
>> --
>> Arun Mathew
>>
>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
>> wrote:
>>
>>     Arun,
>>
>>     I've given you permission to edit the wiki. Let me know if
>> you run
>>
>> into any
>>
>>     issues.
>>
>>     -Ewen
>>
>>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co
>> rp.jp> <am...@yahoo-corp.jp>
>>
>> wrote:
>>
>>
>>     > Thanks Michal. I don’t have the access yet [arunmathew88].
>> Should I
>>
>> be
>>
>>     > sending a separate mail for this?
>>     >
>>     > I thought one of the person following this thread would be
>> able to
>>
>> give me
>>
>>     > access.
>>     >
>>     >
>>     >
>>     > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>     > *Date: *Friday, April 7, 2017 at 17:16
>>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>> semantics
>>     >
>>     >
>>     >
>>     > Hi Arun,
>>     >
>>     > I was thinking along the same lines as you, listing the use
>> cases
>>
>> on the
>>
>>     > wiki, but didn't find time to get around doing that yet.
>>     > Don't mind if you do it if you have access now.
>>     > I was thinking it would be nice if, once we have the use
>> cases
>>
>> listed,
>>
>>     > people could use likes to up-vote the use cases similar to
>> what
>>
>> they're
>>
>>     > working on.
>>     >
>>     > I should have a bit more time to action this in the next
>> few days,
>>
>> but
>>
>>     > happy for you to do it if you can beat me to it ;-)
>>     >
>>     > Cheers,
>>     > Michal
>>     >
>>     > On 07/04/17 04:39, Arun Mathew wrote:
>>     >
>>     > Sure, Thanks Matthias. My id is [arunmathew88].
>>     >
>>     >
>>     >
>>     > Of course. I was thinking of a subpage where people can
>>
>> collaborate.
>>
>>     >
>>     >
>>     >
>>     > Will do as per Michael’s suggestion.
>>     >
>>     >
>>     >
>>     > Regards,
>>     >
>>     > Arun Mathew
>>     >
>>     >
>>     >
>>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
>> <
>>
>> matthias@confluent.io> wrote:
>>
>>     >
>>     >
>>     >
>>     >     Please share your Wiki-ID and a committer can give you
>> write
>>
>> access.
>>
>>     >
>>     >
>>     >
>>     >     Btw: as you did not initiate the KIP, you should not
>> change the
>>
>> KIP
>>
>>     >
>>     >     without the permission of the original author -- in
>> this case
>>
>> Michael.
>>
>>     >
>>     >
>>     >
>>     >     So you might also just share your thought over the
>> mailing list
>>
>> and
>>
>>     >
>>     >     Michael can update the KIP page. Or, as an alternative,
>> just
>>
>> create a
>>
>>     >
>>     >     subpage for the KIP page.
>>     >
>>     >
>>     >
>>     >     @Michael: WDYT?
>>     >
>>     >
>>     >
>>     >
>>     >
>>     >     -Matthias
>>     >
>>     >
>>     >
>>     >
>>     >
>>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>     >
>>     >     > Hi Jay,
>>     >
>>     >     >           Thanks for the advise, I would like to list
>> down
>>
>> the use cases as
>>
>>     >
>>     >     > per your suggestion. But it seems I don't have write
>>
>> permission to the
>>
>>     >
>>     >     > Apache Kafka Confluent Space. Whom shall I request
>> for it?
>>     >
>>     >     >
>>     >
>>     >     > Regarding your last question. We are using a patch in
>> our
>>
>> production system
>>
>>     >
>>     >     > which does exactly this.
>>     >
>>     >     > We window by the event time, but trigger punctuate in
>>
>> <punctuate interval>
>>
>>     >
>>     >     > duration of system time, in the absence of an event
>> crossing
>>
>> the punctuate
>>
>>     >
>>     >     > event time.
>>     >
>>     >     >
>>     >
>>     >     > We are using Kafka Stream for our Audit Trail, where
>> we need
>>
>> to output the
>>
>>     >
>>     >     > event counts on each topic on each cluster aggregated
>> over a
>>
>> 1 minute
>>
>>     >
>>     >     > window. We have to use event time to be able to cross
>> check
>>
>> the counts. But
>>
>>     >
>>     >     > we need to trigger punctuate [aggregate event pushes]
>> by
>>
>> system time in the
>>
>>     >
>>     >     > absence of events. Otherwise the event counts for
>> unexpired
>>
>> windows would
>>
>>     >
>>     >     > be 0 which is bad.
>>     >
>>     >     >
>>     >
>>     >     > "Maybe a hybrid solution works: I window by event
>> time but
>>
>> trigger results
>>
>>     >
>>     >     > by system time for windows that have updated? Not
>> really sure
>>
>> the details
>>
>>     >
>>     >     > of making that work. Does that work? Are there
>> concrete
>>
>> examples where you
>>
>>     >
>>     >     > actually want the current behavior?"
>>     >
>>     >     >
>>     >
>>     >     > --
>>     >
>>     >     > With Regards,
>>     >
>>     >     >
>>     >
>>     >     > Arun Mathew
>>     >
>>     >     > Yahoo! JAPAN Corporation
>>     >
>>     >     >
>>     >
>>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>
>> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>>
>>     >
>>     >     >
>>     >
>>     >     >> Hi Jay,
>>     >
>>     >     >>
>>     >
>>     >     >> The hybrid solution is exactly what I expect and
>> need for
>>
>> our use cases
>>
>>     >
>>     >     >> when dealing with telecom data.
>>     >
>>     >     >>
>>     >
>>     >     >> Thanks
>>     >
>>     >     >> Tianji
>>     >
>>     >     >>
>>     >
>>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>
>> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>>
>>     >
>>     >     >>
>>     >
>>     >     >>> Hey guys,
>>     >
>>     >     >>>
>>     >
>>     >     >>> One thing I've always found super important for
>> this kind
>>
>> of design work
>>
>>     >
>>     >     >> is
>>     >
>>     >     >>> to do a really good job of cataloging the landscape
>> of use
>>
>> cases and how
>>
>>     >
>>     >     >>> prevalent each one is. By that I mean not just
>> listing lots
>>
>> of uses, but
>>
>>     >
>>     >     >>> also grouping them into categories that
>> functionally need
>>
>> the same thing.
>>
>>     >
>>     >     >>> In the absence of this it is very hard to reason
>> about
>>
>> design proposals.
>>
>>     >
>>     >     >>> From the proposals so far I think we have a lot of
>>
>> discussion around
>>
>>     >
>>     >     >>> possible apis, but less around what the user needs
>> for
>>
>> different use
>>
>>     >
>>     >     >> cases
>>     >
>>     >     >>> and how they would implement that using the api.
>>     >
>>     >     >>>
>>     >
>>     >     >>> Here is an example:
>>     >
>>     >     >>> You aggregate click and impression data for a
>> reddit like
>>
>> site. Every ten
>>
>>     >
>>     >     >>> minutes you want to output a ranked list of the top
>> 10
>>
>> articles ranked by
>>
>>     >
>>     >     >>> clicks/impressions for each geographical area. I
>> want to be
>>
>> able run this
>>
>>     >
>>     >     >>> in steady state as well as rerun to regenerate
>> results (or
>>
>> catch up if it
>>
>>     >
>>     >     >>> crashes).
>>     >
>>     >     >>>
>>     >
>>     >     >>> There are a couple of tricky things that seem to
>> make this
>>
>> hard with
>>
>>     >
>>     >     >> either
>>     >
>>     >     >>> of the options proposed:
>>     >
>>     >     >>> 1. If I emit this data using event time I have the
>> problem
>>
>> described
>>
>>     >
>>     >     >> where
>>     >
>>     >     >>> a geographical region with no new clicks or
>> impressions
>>
>> will fail to
>>
>>     >
>>     >     >> output
>>     >
>>     >     >>> results.
>>     >
>>     >     >>> 2. If I emit this data using system time I have the
>> problem
>>
>> that when
>>
>>     >
>>     >     >>> reprocessing data my window may not be ten minutes
>> but 10
>>
>> hours if my
>>
>>     >
>>     >     >>> processing is very fast so it dramatically changes
>> the
>>
>> output.
>>
>>     >
>>     >     >>>
>>     >
>>     >     >>> Maybe a hybrid solution works: I window by event
>> time but
>>
>> trigger results
>>
>>     >
>>     >     >>> by system time for windows that have updated? Not
>> really
>>
>> sure the details
>>
>>     >
>>     >     >>> of making that work. Does that work? Are there
>> concrete
>>
>> examples where
>>
>>     >
>>     >     >> you
>>     >
>>     >     >>> actually want the current behavior?
>>     >
>>     >     >>>
>>     >
>>     >     >>> -Jay
>>     >
>>     >     >>>
>>     >
>>     >     >>>
>>     >
>>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>
>> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>>
>>     >
>>     >     >>> wrote:
>>     >
>>     >     >>>
>>     >
>>     >     >>>> Hi All,
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> Thanks for the KIP. We were also in need of a
>> mechanism to
>>
>> trigger
>>
>>     >
>>     >     >>>> punctuate in the absence of events.
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> As I described in [
>>     >
>>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>     >
>>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>     >
>>     >     >>>> plugin.system.issuetabpanels:comment-
>> tabpanel#comment-
>>
>> 15926036
>>
>>     >
>>     >     >>>> ],
>>     >
>>     >     >>>>
>>     >
>>     >     >>>>    - Our approached involved using the event time
>> by
>>
>> default.
>>
>>     >
>>     >     >>>>    - The method to check if there is any punctuate
>> ready
>>
>> in the
>>
>>     >
>>     >     >>>>    PunctuationQueue is triggered via the any event
>>
>> received by the
>>
>>     >
>>     >     >> stream
>>     >
>>     >     >>>>    tread, or at the polling intervals in the
>> absence of
>>
>> any events.
>>
>>     >
>>     >     >>>>    - When we create Punctuate objects (which
>> contains the
>>
>> next event
>>
>>     >
>>     >     >> time
>>     >
>>     >     >>>>    for punctuation and interval), we also record
>> the
>>
>> creation time
>>
>>     >
>>     >     >>> (system
>>     >
>>     >     >>>>    time).
>>     >
>>     >     >>>>    - While checking for maturity of Punctuate
>> Schedule by
>>     >
>>     >     >> mayBePunctuate
>>     >
>>     >     >>>>    method, we also check if the system clock has
>> elapsed
>>
>> the punctuate
>>
>>     >
>>     >     >>>>    interval since the schedule creation time.
>>     >
>>     >     >>>>    - In the absence of any event, or in the
>> absence of any
>>
>> event for
>>
>>     >
>>     >     >> one
>>     >
>>     >     >>>>    topic in the partition group assigned to the
>> stream
>>
>> task, the system
>>
>>     >
>>     >     >>>> time
>>     >
>>     >     >>>>    will elapse the interval and we trigger a
>> punctuate
>>
>> using the
>>
>>     >
>>     >     >> expected
>>     >
>>     >     >>>>    punctuation event time.
>>     >
>>     >     >>>>    - we then create the next punctuation schedule
>> as
>>
>> punctuation event
>>
>>     >
>>     >     >>> time
>>     >
>>     >     >>>>    + punctuation interval, [again recording the
>> system
>>
>> time of creation
>>
>>     >
>>     >     >>> of
>>     >
>>     >     >>>> the
>>     >
>>     >     >>>>    schedule].
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> We call this a Hybrid Punctuate. Of course, this
>> approach
>>
>> has pros and
>>
>>     >
>>     >     >>>> cons.
>>     >
>>     >     >>>> Pros
>>     >
>>     >     >>>>
>>     >
>>     >     >>>>    - Punctuates will happen in <punctuate
>> interval> time
>>
>> duration at
>>
>>     >
>>     >     >> max
>>     >
>>     >     >>> in
>>     >
>>     >     >>>>    terms of system time.
>>     >
>>     >     >>>>    - The semantics as a whole continues to revolve
>> around
>>
>> event time.
>>
>>     >
>>     >     >>>>    - We can use the old data [old timestamps] to
>> rerun any
>>
>> experiments
>>
>>     >
>>     >     >> or
>>     >
>>     >     >>>>    tests.
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> Cons
>>     >
>>     >     >>>>
>>     >
>>     >     >>>>    - In case the  <punctuate interval> is not a
>> time
>>
>> duration [say
>>
>>     >
>>     >     >>> logical
>>     >
>>     >     >>>>    time/event count], then the approach might not
>> be
>>
>> meaningful.
>>
>>     >
>>     >     >>>>    - In case there is a case where we have to wait
>> for an
>>
>> actual event
>>
>>     >
>>     >     >>> from
>>     >
>>     >     >>>>    a low event rate partition in the partition
>> group, this
>>
>> approach
>>
>>     >
>>     >     >> will
>>     >
>>     >     >>>> jump
>>     >
>>     >     >>>>    the gun.
>>     >
>>     >     >>>>    - in case the event processing cannot catch up
>> with the
>>
>> event rate
>>
>>     >
>>     >     >> and
>>     >
>>     >     >>>>    the expected timestamp events gets queued for
>> long
>>
>> time, this
>>
>>     >
>>     >     >> approach
>>     >
>>     >     >>>>    might jump the gun.
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> I believe the above approach and discussion goes
>> close to
>>
>> the approach
>>
>>     >
>>     >     >> A.
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> -----------
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> I like the idea of having an even count based
>> punctuate.
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> -----------
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> I agree with the discussion around approach C,
>> that we
>>
>> should provide
>>
>>     >
>>     >     >> the
>>     >
>>     >     >>>> user with the option to choose system time or
>> event time
>>
>> based
>>
>>     >
>>     >     >>> punctuates.
>>     >
>>     >     >>>> But I believe that the user predominantly wants to
>> use
>>
>> event time while
>>
>>     >
>>     >     >>> not
>>     >
>>     >     >>>> missing out on regular punctuates due to event
>> delays or
>>
>> event
>>
>>     >
>>     >     >> absences.
>>     >
>>     >     >>>> Hence a complex punctuate option as Matthias
>> mentioned
>>
>> (quoted below)
>>
>>     >
>>     >     >>> would
>>     >
>>     >     >>>> be most apt.
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> "- We might want to add "complex" schedules later
>> on
>>
>> (like, punctuate
>>
>>     >
>>     >     >> on
>>     >
>>     >     >>>> every 10 seconds event-time or 60 seconds system-
>> time
>>
>> whatever comes
>>
>>     >
>>     >     >>>> first)."
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> -----------
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> I think I read somewhere that Kafka Streams
>> started with
>>
>> System Time as
>>
>>     >
>>     >     >>> the
>>     >
>>     >     >>>> punctuation standard, but was later changed to
>> Event Time.
>>
>> I guess
>>
>>     >
>>     >     >> there
>>     >
>>     >     >>>> would be some good reason behind it. As Kafka
>> Streams want
>>
>> to evolve
>>
>>     >
>>     >     >> more
>>     >
>>     >     >>>> on the Stream Processing front, I believe the
>> emphasis on
>>
>> event time
>>
>>     >
>>     >     >>> would
>>     >
>>     >     >>>> remain quite strong.
>>     >
>>     >     >>>>
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> With Regards,
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> Arun Mathew
>>     >
>>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>     >
>>     >     >>>>
>>     >
>>     >     >>>>
>>     >
>>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>
>> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>>
>>     >
>>     >     >> wrote:
>>     >
>>     >     >>>>
>>     >
>>     >     >>>>> Yeah I like PuncutationType much better; I just
>> threw
>>
>> Time out there
>>
>>     >
>>     >     >>>>> more as a strawman than an actual suggestion ;) I
>> still
>>
>> think it's
>>
>>     >
>>     >     >>>>> worth considering what this buys us over an
>> additional
>>
>> callback. I
>>
>>     >
>>     >     >>>>> foresee a number of punctuate implementations
>> following
>>
>> this pattern:
>>
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>> public void punctuate(PunctuationType type) {
>>     >
>>     >     >>>>>     switch (type) {
>>     >
>>     >     >>>>>         case EVENT_TIME:
>>     >
>>     >     >>>>>             methodA();
>>     >
>>     >     >>>>>             break;
>>     >
>>     >     >>>>>         case SYSTEM_TIME:
>>     >
>>     >     >>>>>             methodB();
>>     >
>>     >     >>>>>             break;
>>     >
>>     >     >>>>>     }
>>     >
>>     >     >>>>> }
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>> I guess one advantage of this approach is we
>> could add
>>
>> additional
>>
>>     >
>>     >     >>>>> punctuation types later in a backwards compatible
>> way
>>
>> (like event
>>
>>     >
>>     >     >> count
>>     >
>>     >     >>>>> as you mentioned).
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>> -Tommy
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>> Sax wrote:
>>     >
>>     >     >>>>>> That sounds promising.
>>     >
>>     >     >>>>>>
>>     >
>>     >     >>>>>> I am just wondering if `Time` is the best name.
>> Maybe we
>>
>> want to
>>
>>     >
>>     >     >> add
>>     >
>>     >     >>>>>> other non-time based punctuations at some point
>> later. I
>>
>> would
>>
>>     >
>>     >     >>>>>> suggest
>>     >
>>     >     >>>>>>
>>     >
>>     >     >>>>>> enum PunctuationType {
>>     >
>>     >     >>>>>>   EVENT_TIME,
>>     >
>>     >     >>>>>>   SYSTEM_TIME,
>>     >
>>     >     >>>>>> }
>>     >
>>     >     >>>>>>
>>     >
>>     >     >>>>>> or similar. Just to keep the door open -- it's
>> easier to
>>
>> add new
>>
>>     >
>>     >     >>>>>> stuff
>>     >
>>     >     >>>>>> if the name is more generic.
>>     >
>>     >     >>>>>>
>>     >
>>     >     >>>>>>
>>     >
>>     >     >>>>>> -Matthias
>>     >
>>     >     >>>>>>
>>     >
>>     >     >>>>>>
>>     >
>>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>> I agree that the framework providing and
>> managing the
>>
>> notion of
>>
>>     >
>>     >     >>>>>>> stream
>>     >
>>     >     >>>>>>> time is valuable and not something we would
>> want to
>>
>> delegate to
>>
>>     >
>>     >     >> the
>>     >
>>     >     >>>>>>> tasks. I'm not entirely convinced that a
>> separate
>>
>> callback
>>
>>     >
>>     >     >> (option
>>     >
>>     >     >>>>>>> C)
>>     >
>>     >     >>>>>>> is that messy (it could just be a default
>> method with
>>
>> an empty
>>
>>     >
>>     >     >>>>>>> implementation), but if we wanted a single API
>> to
>>
>> handle both
>>
>>     >
>>     >     >>>>>>> cases,
>>     >
>>     >     >>>>>>> how about something like the following?
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>> enum Time {
>>     >
>>     >     >>>>>>>    STREAM,
>>     >
>>     >     >>>>>>>    CLOCK
>>     >
>>     >     >>>>>>> }
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>> Then on ProcessorContext:
>>     >
>>     >     >>>>>>> context.schedule(Time time, long interval)  //
>> We could
>>
>> allow
>>
>>     >
>>     >     >> this
>>     >
>>     >     >>>>>>> to
>>     >
>>     >     >>>>>>> be called once for each value of time to mix
>>
>> approaches.
>>
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>> Then the Processor API becomes:
>>     >
>>     >     >>>>>>> punctuate(Time time) // time here denotes which
>>
>> schedule resulted
>>
>>     >
>>     >     >>>>>>> in
>>     >
>>     >     >>>>>>> this call.
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>> Thoughts?
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>> Sax
>>
>> wrote:
>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> I was thinking about the four options you
>> proposed in
>>
>> more
>>
>>     >
>>     >     >>>>>>>> details
>>     >
>>     >     >>>>>>>> and
>>     >
>>     >     >>>>>>>> this are my thoughts:
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> (A) You argue, that users can still
>> "punctuate" on
>>
>> event-time
>>
>>     >
>>     >     >> via
>>     >
>>     >     >>>>>>>> process(), but I am not sure if this is
>> possible.
>>
>> Note, that
>>
>>     >
>>     >     >>>>>>>> users
>>     >
>>     >     >>>>>>>> only
>>     >
>>     >     >>>>>>>> get record timestamps via context.timestamp().
>> Thus,
>>
>> users
>>
>>     >
>>     >     >> would
>>     >
>>     >     >>>>>>>> need
>>     >
>>     >     >>>>>>>> to
>>     >
>>     >     >>>>>>>> track the time progress per partition (based
>> on the
>>
>> partitions
>>
>>     >
>>     >     >>>>>>>> they
>>     >
>>     >     >>>>>>>> obverse via context.partition(). (This alone
>> puts a
>>
>> huge burden
>>
>>     >
>>     >     >>>>>>>> on
>>     >
>>     >     >>>>>>>> the
>>     >
>>     >     >>>>>>>> user by itself.) However, users are not
>> notified at
>>
>> startup
>>
>>     >
>>     >     >> what
>>     >
>>     >     >>>>>>>> partitions are assigned, and user are not
>> notified
>>
>> when
>>
>>     >
>>     >     >>>>>>>> partitions
>>     >
>>     >     >>>>>>>> get
>>     >
>>     >     >>>>>>>> revoked. Because this information is not
>> available,
>>
>> it's not
>>
>>     >
>>     >     >>>>>>>> possible
>>     >
>>     >     >>>>>>>> to
>>     >
>>     >     >>>>>>>> "manually advance" stream-time, and thus
>> event-time
>>
>> punctuation
>>
>>     >
>>     >     >>>>>>>> within
>>     >
>>     >     >>>>>>>> process() seems not to be possible -- or do
>> you see a
>>
>> way to
>>
>>     >
>>     >     >> get
>>     >
>>     >     >>>>>>>> it
>>     >
>>     >     >>>>>>>> done? And even if, it might still be too
>> clumsy to
>>
>> use.
>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> (B) This does not allow to mix both
>> approaches, thus
>>
>> limiting
>>
>>     >
>>     >     >>>>>>>> what
>>     >
>>     >     >>>>>>>> users
>>     >
>>     >     >>>>>>>> can do.
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> (C) This should give all flexibility we need.
>> However,
>>
>> just
>>
>>     >
>>     >     >>>>>>>> adding
>>     >
>>     >     >>>>>>>> one
>>     >
>>     >     >>>>>>>> more method seems to be a solution that is too
>> simple
>>
>> (cf my
>>
>>     >
>>     >     >>>>>>>> comments
>>     >
>>     >     >>>>>>>> below).
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> (D) This might be hard to use. Also, I am not
>> sure how
>>
>> a user
>>
>>     >
>>     >     >>>>>>>> could
>>     >
>>     >     >>>>>>>> enable system-time and event-time punctuation
>> in
>>
>> parallel.
>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> Overall options (C) seems to be the most
>> promising
>>
>> approach to
>>
>>     >
>>     >     >>>>>>>> me.
>>     >
>>     >     >>>>>>>> Because I also favor a clean API, we might
>> keep
>>
>> current
>>
>>     >
>>     >     >>>>>>>> punctuate()
>>     >
>>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>> at some
>>
>> later
>>
>>     >
>>     >     >>>>>>>> point
>>     >
>>     >     >>>>>>>> when
>>     >
>>     >     >>>>>>>> people use the "new punctuate API".
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> Couple of follow up questions:
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> - I am wondering, if we should have two
>> callback
>>
>> methods or
>>
>>     >
>>     >     >> just
>>     >
>>     >     >>>>>>>> one
>>     >
>>     >     >>>>>>>> (ie, a unified for system and event time
>> punctuation
>>
>> or one for
>>
>>     >
>>     >     >>>>>>>> each?).
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> - If we have one, how can the user figure out,
>> which
>>
>> condition
>>
>>     >
>>     >     >>>>>>>> did
>>     >
>>     >     >>>>>>>> trigger?
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> - How would the API look like, for registering
>>
>> different
>>
>>     >
>>     >     >>>>>>>> punctuate
>>     >
>>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> - We might want to add "complex" schedules
>> later on
>>
>> (like,
>>
>>     >
>>     >     >>>>>>>> punctuate
>>     >
>>     >     >>>>>>>> on
>>     >
>>     >     >>>>>>>> every 10 seconds event-time or 60 seconds
>> system-time
>>
>> whatever
>>
>>     >
>>     >     >>>>>>>> comes
>>     >
>>     >     >>>>>>>> first). I don't say we should add this right
>> away, but
>>
>> we might
>>
>>     >
>>     >     >>>>>>>> want
>>     >
>>     >     >>>>>>>> to
>>     >
>>     >     >>>>>>>> define the API in a way, that it allows
>> extensions
>>
>> like this
>>
>>     >
>>     >     >>>>>>>> later
>>     >
>>     >     >>>>>>>> on,
>>     >
>>     >     >>>>>>>> without redesigning the API (ie, the API
>> should be
>>
>> designed
>>
>>     >
>>     >     >>>>>>>> extensible)
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> - Did you ever consider count-based
>> punctuation?
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> I understand, that you would like to solve a
>> simple
>>
>> problem,
>>
>>     >
>>     >     >> but
>>     >
>>     >     >>>>>>>> we
>>     >
>>     >     >>>>>>>> learned from the past, that just "adding some
>> API"
>>
>> quickly
>>
>>     >
>>     >     >> leads
>>     >
>>     >     >>>>>>>> to a
>>     >
>>     >     >>>>>>>> not very well defined API that needs time
>> consuming
>>
>> clean up
>>
>>     >
>>     >     >>>>>>>> later on
>>     >
>>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>> holistic
>>     >
>>     >     >>>>>>>> punctuation
>>     >
>>     >     >>>>>>>> KIP
>>     >
>>     >     >>>>>>>> with this from the beginning on to avoid later
>> painful
>>     >
>>     >     >> redesign.
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> -Matthias
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>>
>>     >
>>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>> Thanks Thomas,
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>> I'm also wary of changing the existing
>> semantics of
>>     >
>>     >     >> punctuate,
>>     >
>>     >     >>>>>>>>> for
>>     >
>>     >     >>>>>>>>> backward compatibility reasons, although I
>> like the
>>     >
>>     >     >> conceptual
>>     >
>>     >     >>>>>>>>> simplicity of that option.
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>> Adding a new method to me feels safer but, in
>> a way,
>>
>> uglier.
>>
>>     >
>>     >     >> I
>>     >
>>     >     >>>>>>>>> added
>>     >
>>     >     >>>>>>>>> this to the KIP now as option (C).
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>> The TimestampExtractor mechanism is actually
>> more
>>
>> flexible,
>>
>>     >
>>     >     >> as
>>     >
>>     >     >>>>>>>>> it
>>     >
>>     >     >>>>>>>>> allows
>>     >
>>     >     >>>>>>>>> you to return any value, you're not limited
>> to event
>>
>> time or
>>
>>     >
>>     >     >>>>>>>>> system
>>     >
>>     >     >>>>>>>>> time
>>     >
>>     >     >>>>>>>>> (although I don't see an actual use case
>> where you
>>
>> might need
>>
>>     >
>>     >     >>>>>>>>> anything
>>     >
>>     >     >>>>>>>>> else then those two). Hence I also proposed
>> the
>>
>> option to
>>
>>     >
>>     >     >> allow
>>     >
>>     >     >>>>>>>>> users
>>     >
>>     >     >>>>>>>>> to, effectively, decide what "stream time" is
>> for
>>
>> them given
>>
>>     >
>>     >     >>>>>>>>> the
>>     >
>>     >     >>>>>>>>> presence or absence of messages, much like
>> they can
>>
>> decide
>>
>>     >
>>     >     >> what
>>     >
>>     >     >>>>>>>>> msg
>>     >
>>     >     >>>>>>>>> time
>>     >
>>     >     >>>>>>>>> means for them using the TimestampExtractor.
>> What do
>>
>> you
>>
>>     >
>>     >     >> think
>>     >
>>     >     >>>>>>>>> about
>>     >
>>     >     >>>>>>>>> that? This is probably most flexible but also
>> most
>>     >
>>     >     >> complicated.
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>> All comments appreciated.
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>> Cheers,
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>> Michal
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>>
>>     >
>>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>     >
>>     >     >>>>>>>>>>
>>     >
>>     >     >>>>>>>>>>
>>     >
>>     >     >>>>>>>>>> Although I fully agree we need a way to
>> trigger
>>
>> periodic
>>
>>     >
>>     >     >>>>>>>>>> processing
>>     >
>>     >     >>>>>>>>>> that is independent from whether and when
>> messages
>>
>> arrive,
>>
>>     >
>>     >     >>>>>>>>>> I'm
>>     >
>>     >     >>>>>>>>>> not sure
>>     >
>>     >     >>>>>>>>>> I like the idea of changing the existing
>> semantics
>>
>> across
>>
>>     >
>>     >     >> the
>>     >
>>     >     >>>>>>>>>> board.
>>     >
>>     >     >>>>>>>>>> What if we added an additional callback to
>> Processor
>>
>> that
>>
>>     >
>>     >     >> can
>>     >
>>     >     >>>>>>>>>> be
>>     >
>>     >     >>>>>>>>>> scheduled similarly to punctuate() but was
>> always
>>
>> called at
>>
>>     >
>>     >     >>>>>>>>>> fixed, wall
>>     >
>>     >     >>>>>>>>>> clock based intervals? This way you wouldn't
>> have to
>>
>> give
>>
>>     >
>>     >     >> up
>>     >
>>     >     >>>>>>>>>> the
>>     >
>>     >     >>>>>>>>>> notion
>>     >
>>     >     >>>>>>>>>> of stream time to be able to do periodic
>> processing.
>>     >
>>     >     >>>>>>>>>>
>>     >
>>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>> Borowiecki
>>
>> wrote:
>>
>>     >
>>     >     >>>>>>>>>>>
>>     >
>>     >     >>>>>>>>>>>
>>     >
>>     >     >>>>>>>>>>> Hi all,
>>     >
>>     >     >>>>>>>>>>>
>>     >
>>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>
>> punctuate
>>
>>     >
>>     >     >>>>>>>>>>> semantics
>>     >
>>     >     >>>>>>>>>>> <https://cwiki.apache.org/
>>
>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
>> confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>
>>     >
>>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI
>> P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>
>> 138%
>>
>>     >
>>     >     >>>>>>>>>>> 3A+C
>>     >
>>     >     >>>>>>>>>>> hange+
>>     >
>>     >     >>>>>>>>>>> punctuate+semantics>
>>     >
>>     >     >>>>>>>>>>> .
>>     >
>>     >     >>>>>>>>>>>
>>     >
>>     >     >>>>>>>>>>> Appreciating there can be different views
>> on
>>
>> system-time
>>
>>     >
>>     >     >> vs
>>     >
>>     >     >>>>>>>>>>> event-
>>     >
>>     >     >>>>>>>>>>> time
>>     >
>>     >     >>>>>>>>>>> semantics for punctuation depending on use-
>> case and
>>
>> the
>>
>>     >
>>     >     >>>>>>>>>>> importance of
>>     >
>>     >     >>>>>>>>>>> backwards compatibility of any such change,
>> I've
>>
>> left it
>>
>>     >
>>     >     >>>>>>>>>>> quite
>>     >
>>     >     >>>>>>>>>>> open
>>     >
>>     >     >>>>>>>>>>> and
>>     >
>>     >     >>>>>>>>>>> hope to fill in more info as the discussion
>>
>> progresses.
>>
>>     >
>>     >     >>>>>>>>>>>
>>     >
>>     >     >>>>>>>>>>> Thanks,
>>     >
>>     >     >>>>>>>>>>> Michal
>>     >
>>     >     >>>>>>> --
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>>     Tommy Becker
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>>     Senior Software Engineer
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>>     tivo.com
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>> ________________________________
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>>>> This email and any attachments may contain
>> confidential
>>
>> and
>>
>>     >
>>     >     >>>>>>> privileged material for the sole use of the
>> intended
>>
>> recipient.
>>
>>     >
>>     >     >> Any
>>     >
>>     >     >>>>>>> review, copying, or distribution of this email
>> (or any
>>     >
>>     >     >> attachments)
>>     >
>>     >     >>>>>>> by others is prohibited. If you are not the
>> intended
>>
>> recipient,
>>
>>     >
>>     >     >>>>>>> please contact the sender immediately and
>> permanently
>>
>> delete this
>>
>>     >
>>     >     >>>>>>> email and any attachments. No employee or agent
>> of TiVo
>>
>> Inc. is
>>
>>     >
>>     >     >>>>>>> authorized to conclude any binding agreement on
>> behalf
>>
>> of TiVo
>>
>>     >
>>     >     >> Inc.
>>     >
>>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>> only be
>>
>> made by a
>>
>>     >
>>     >     >>>>>>> signed written agreement.
>>     >
>>     >     >>>>>>>
>>     >
>>     >     >>>>> --
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>>     Tommy Becker
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>>     Senior Software Engineer
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>>     O +1 919.460.4747 <%28919%29%20460-4747> <(919)%20460-4747>
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>>     tivo.com
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>> ________________________________
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>> This email and any attachments may contain
>> confidential
>>
>> and
>>
>>     >
>>     >     >> privileged
>>     >
>>     >     >>>>> material for the sole use of the intended
>> recipient. Any
>>
>> review,
>>
>>     >
>>     >     >>> copying,
>>     >
>>     >     >>>>> or distribution of this email (or any
>> attachments) by
>>
>> others is
>>
>>     >
>>     >     >>>> prohibited.
>>     >
>>     >     >>>>> If you are not the intended recipient, please
>> contact the
>>
>> sender
>>
>>     >
>>     >     >>>>> immediately and permanently delete this email and
>> any
>>
>> attachments. No
>>
>>     >
>>     >     >>>>> employee or agent of TiVo Inc. is authorized to
>> conclude
>>
>> any binding
>>
>>     >
>>     >     >>>>> agreement on behalf of TiVo Inc. by email.
>> Binding
>>
>> agreements with
>>
>>     >
>>     >     >> TiVo
>>     >
>>     >     >>>>> Inc. may only be made by a signed written
>> agreement.
>>     >
>>     >     >>>>>
>>     >
>>     >     >>>>
>>     >
>>     >     >>>
>>     >
>>     >     >>
>>     >
>>     >     >
>>     >
>>     >
>>     >
>>     >
>>     >
>>     >
>>     >
>>     >
>>     >
>>     > --
>>     >
>>     > <http://www.openbet.com/> <http://www.openbet.com/>
>>
>>     >
>>     > *Michal Borowiecki*
>>     >
>>     > *Senior Software Engineer L4*
>>     >
>>     > *T: *
>>     >
>>     > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>>     >
>>     > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>>     >
>>     >
>>     >
>>     > *E: *
>>     >
>>     > michal.borowiecki@openbet.com
>>     >
>>     > *W: *
>>     >
>>     > www.openbet.com
>>     >
>>     > *OpenBet Ltd*
>>     >
>>     > Chiswick Park Building 9
>>     >
>>     > 566 Chiswick High Rd
>>     >
>>     > London
>>     >
>>     > W4 5XT
>>     >
>>     > UK
>>     >
>>     > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>>     >
>>     >
>>     >
>>     > This message is confidential and intended only for the
>> addressee.
>>
>> If you
>>
>>     > have received this message in error, please immediately
>> notify the
>>     > postmaster@openbet.com and delete it from your system as
>> well as
>>
>> any
>>
>>     > copies. The content of e-mails as well as traffic data may
>> be
>>
>> monitored by
>>
>>     > OpenBet for employment and security purposes. To protect
>> the
>>
>> environment
>>
>>     > please do not print this e-mail unless necessary. OpenBet
>> Ltd.
>>
>> Registered
>>
>>     > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>> London,
>>
>> W4 5XT,
>>
>>     > United Kingdom. A company registered in England and Wales.
>>
>> Registered no.
>>
>>     > 3134634. VAT no. GB927523612
>>     >
>>     >
>>     >
>>
>>
>>
>> --
>>
>>
>>     Tommy Becker
>>
>>     Senior Software Engineer
>>
>>     O +1 919.460.4747 <%28919%29%20460-4747>
>>
>>
>>     tivo.com
>>
>>
>>
>> ________________________________
>>
>> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>>
>>
>> --
>> <http://www.openbet.com/> Michal Borowiecki
>> Senior Software Engineer L4
>> T: +44 208 742 1600 <+44%2020%208742%201600>
>>
>>
>>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Yes, that looks better to me.

Note that punctuate on Transformer is currently returning a record, but 
I think it's ok to have all output records be sent via 
ProcessorContext.forward, which has to be used anyway if you want to 
send multiple records from one invocation of punctuate.

This way it's consistent between Processor and Transformer.


BTW, looking at this I found a glitch in the javadoc and put a comment 
there:

https://github.com/apache/kafka/pull/2413/files#r112634612

and PR: https://github.com/apache/kafka/pull/2884

Cheers,

Michal

On 20/04/17 18:55, Damian Guy wrote:
> Hi Michal,
>
> Thanks for the KIP. I'd like to propose a bit more of a radical change 
> to the API.
> 1. deprecate the punctuate method on Processor
> 2. create a new Functional Interface just for Punctuation, something like:
> interface Punctuator {
>     void punctuate(long timestamp)
> }
> 3. add a new schedule function to ProcessorContext: schedule(long 
> interval, PunctuationType type, Punctuator callback)
> 4. deprecate the existing schedule function
>
> Thoughts?
>
> Thanks,
> Damian
>
> On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki 
> <michal.borowiecki@openbet.com <ma...@openbet.com>> 
> wrote:
>
>     Hi Thomas,
>
>     I would say our use cases fall in the same category as yours.
>
>     1) One is expiry of old records, it's virtually identical to yours.
>
>     2) Second one is somewhat more convoluted but boils down to the
>     same type of design:
>
>     Incoming messages carry a number of fields, including a timestamp.
>
>     Outgoing messages contain derived fields, one of them (X) is
>     depended on by the timestamp input field (Y) and some other input
>     field (Z).
>
>     Since the output field X is derived in some non-trivial way, we
>     don't want to force the logic onto downstream apps. Instead we
>     want to calculate it in the Kafka Streams app, which means we
>     re-calculate X as soon as the timestamp in Y is reached (wall
>     clock time) and send a message if it changed (I say "if" because
>     the derived field (X) is also conditional on another input field Z).
>
>
>     So we have kv stores with the records and an additional kv store
>     with timestamp->id mapping which act like an index where we
>     periodically do a ranged query.
>
>     Initially we naively tried doing it in punctuate which of course
>     didn't work when there were no regular msgs on the input topic.
>     Since this was before 0.10.1 and state stores weren't query-able
>     from outside we created a "ticker" that produced msgs once per
>     second onto another topic and fed it into the same topology to
>     trigger punctuate.
>     This didn't work either, which was much more surprising to us at
>     the time, because it was not obvious at all that punctuate is only
>     triggered if /*all*/ input partitions receive messages regularly.
>     In the end we had to break this into 2 separate Kafka Streams.
>     Main transformer doesn't use punctuate but sends values of
>     timestamp field Y and the id to a "scheduler" topic where also the
>     periodic ticks are sent. This is consumed by the second topology
>     and is its only input topic. There's a transformer on that topic
>     which populates and updates the time-based indexes and polls them
>     from punctuate. If the time in the timestamp elapsed, the record
>     id is sent to the main transformer, which updates/deletes the
>     record from the main kv store and forwards the transformed record
>     to the output topic.
>
>     To me this setup feels horrendously complicated for what it does.
>
>     We could incrementally improve on this since 0.10.1 to poll the
>     timestamp->id "index" stores from some code outside the
>     KafkaStreams topology so that at least we wouldn't need the extra
>     topic for "ticks".
>     However, the ticks don't feel so hacky when you realise they give
>     you some hypothetical benefits in predictability. You can
>     reprocess the messages in a reproducible manner, since the
>     topologies use event-time, just that the event time is simply the
>     wall-clock time fed into a topic by the ticks. (NB in our use case
>     we haven't yet found a need for this kind of reprocessing).
>     To make that work though, we would have to have the stream time
>     advance based on the presence of msgs on the "tick" topic,
>     regardless of the presence of messages on the other input topic.
>
>     Same as in the expiry use case, both the wall-clock triggered
>     punctuate and the hybrid would work to simplify this a lot.
>
>     3) Finally, I have a 3rd use case in the making but I'm still
>     looking if we can achieve it using session windows instead. I'll
>     keep you posted if we have to go with punctuate there too.
>
>     Thanks,
>     Michal
>
>
>     On 11/04/17 20:52, Thomas Becker wrote:
>>     Here's an example that we currently have.  We have a streams processor
>>     that does a transform from one topic into another. One of the fields in
>>     the source topic record is an expiration time, and one of the functions
>>     of the processor is to ensure that expired records get deleted promptly
>>     after that time passes (typically days or weeks after the message was
>>     originally produced). To do that, the processor keeps a state store of
>>     keys and expiration times, iterates that store in punctuate(), and
>>     emits delete (null) records for expired items. This needs to happen at
>>     some minimum interval regardless of the incoming message rate of the
>>     source topic.
>>
>>     In this scenario, the expiration of records is the primary function of
>>     punctuate, and therefore the key requirement is that the wall-clock
>>     measured time between punctuate calls have some upper-bound. So a pure
>>     wall-clock based schedule would be fine for our needs. But the proposed
>>     "hybrid" system would also be acceptable if that satisfies a broader
>>     range of use-cases.
>>
>>     On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>>>     I apologize for the longer email below.  To my defense, it started
>>>     out much
>>>     shorter. :-)  Also, to be super-clear, I am intentionally playing
>>>     devil's
>>>     advocate for a number of arguments brought forth in order to help
>>>     improve
>>>     this KIP -- I am not implying I necessarily disagree with the
>>>     arguments.
>>>
>>>     That aside, here are some further thoughts.
>>>
>>>     First, there are (at least?) two categories for actions/behavior you
>>>     invoke
>>>     via punctuate():
>>>
>>>     1. For internal housekeeping of your Processor or Transformer (e.g.,
>>>     to
>>>     periodically commit to a custom store, to do metrics/logging).  Here,
>>>     the
>>>     impact of punctuate is typically not observable by other processing
>>>     nodes
>>>     in the topology.
>>>     2. For controlling the emit frequency of downstream records.  Here,
>>>     the
>>>     punctuate is all about being observable by downstream processing
>>>     nodes.
>>>
>>>     A few releases back, we introduced record caches (DSL) and state
>>>     store
>>>     caches (Processor API) in KIP-63.  Here, we addressed a concern
>>>     relating to
>>>     (2) where some users needed to control -- here: limit -- the
>>>     downstream
>>>     output rate of Kafka Streams because the downstream systems/apps
>>>     would not
>>>     be able to keep up with the upstream output rate (Kafka scalability >
>>>     their
>>>     scalability).  The argument for KIP-63, which notably did not
>>>     introduce a
>>>     "trigger" API, was that such an interaction with downstream systems
>>>     is an
>>>     operational concern;  it should not impact the processing *logic* of
>>>     your
>>>     application, and thus we didn't want to complicate the Kafka Streams
>>>     API,
>>>     especially not the declarative DSL, with such operational concerns.
>>>
>>>     This KIP's discussion on `punctuate()` takes us back in time (<--
>>>     sorry, I
>>>     couldn't resist to not make this pun :-P).  As a meta-comment, I am
>>>     observing that our conversation is moving more and more into the
>>>     direction
>>>     of explicit "triggers" because, so far, I have seen only motivations
>>>     for
>>>     use cases in category (2), but none yet for (1)?  For example, some
>>>     comments voiced here are about sth like "IF stream-time didn't
>>>     trigger
>>>     punctuate, THEN trigger punctuate based on processing-time".  Do we
>>>     want
>>>     this, and if so, for which use cases and benefits?  Also, on a
>>>     related
>>>     note, whatever we are discussing here will impact state store caches
>>>     (Processor API) and perhaps also impact record caches (DSL), thus we
>>>     should
>>>     clarify any such impact here.
>>>
>>>     Switching topics slightly.
>>>
>>>     Jay wrote:
>>>>     One thing I've always found super important for this kind of design
>>>>     work
>>>>     is to do a really good job of cataloging the landscape of use cases
>>>>     and
>>>>     how prevalent each one is.
>>>     +1 to this, as others have already said.
>>>
>>>     Here, let me highlight -- just in case -- that when we talked about
>>>     windowing use cases in the recent emails, the Processor API (where
>>>     `punctuate` resides) does not have any notion of windowing at
>>>     all.  If you
>>>     want to do windowing *in the Processor API*, you must do so manually
>>>     in
>>>     combination with window stores.  For this reason I'd suggest to
>>>     discuss use
>>>     cases not just in general, but also in view of how you'd do so in the
>>>     Processor API vs. in the DSL.  Right now, changing/improving
>>>     `punctuate`
>>>     does not impact the DSL at all, unless we add new functionality to
>>>     it.
>>>
>>>     Jay wrote in his strawman example:
>>>>     You aggregate click and impression data for a reddit like site.
>>>>     Every ten
>>>>     minutes you want to output a ranked list of the top 10 articles
>>>>     ranked by
>>>>     clicks/impressions for each geographical area. I want to be able
>>>>     run this
>>>>     in steady state as well as rerun to regenerate results (or catch up
>>>>     if it
>>>>     crashes).
>>>     This is a good example for more than the obvious reason:  In KIP-63,
>>>     we
>>>     argued that the reason for saying "every ten minutes" above is not
>>>     necessarily about because you want to output data *exactly* after ten
>>>     minutes, but that you want to perform an aggregation based on 10-
>>>     minute
>>>     windows of input data; i.e., the point is about specifying the input
>>>     for
>>>     your aggregation, not or less about when the results of the
>>>     aggregation
>>>     should be send downstream.  To take an extreme example, you could
>>>     disable
>>>     record caches and let your app output a downstream update for every
>>>     incoming input record.  If the last input record was from at minute 7
>>>     of 10
>>>     (for a 10-min window), then what your app would output at minute 10
>>>     would
>>>     be identical to what it had already emitted at minute 7 earlier
>>>     anyways.
>>>     This is particularly true when we take late-arriving data into
>>>     account:  if
>>>     a late record arrived at minute 13, your app would (by default) send
>>>     a new
>>>     update downstream, even though the "original" 10 minutes have already
>>>     passed.
>>>
>>>     Jay wrote...:
>>>>     There are a couple of tricky things that seem to make this hard
>>>>     with
>>>     either
>>>>     of the options proposed:
>>>>     1. If I emit this data using event time I have the problem
>>>>     described where
>>>>     a geographical region with no new clicks or impressions will fail
>>>>     to
>>>     output
>>>>     results.
>>>     ...and Arun Mathew wrote:
>>>
>>>>     We window by the event time, but trigger punctuate in <punctuate
>>>>     interval>
>>>>     duration of system time, in the absence of an event crossing the
>>>>     punctuate
>>>>     event time.
>>>     So, given what I wrote above about the status quo and what you can
>>>     already
>>>     do with it, is the concern that the state store cache doesn't give
>>>     you
>>>     *direct* control over "forcing an output after no later than X
>>>     seconds [of
>>>     processing-time]" but only indirect control through a cache
>>>     size?  (Note
>>>     that I am not dismissing the claims why this might be helpful.)
>>>
>>>     Arun Mathew wrote:
>>>>     We are using Kafka Stream for our Audit Trail, where we need to
>>>>     output the
>>>>     event counts on each topic on each cluster aggregated over a 1
>>>>     minute
>>>>     window. We have to use event time to be able to cross check the
>>>>     counts.
>>>     But
>>>>     we need to trigger punctuate [aggregate event pushes] by system
>>>>     time in
>>>     the
>>>>     absence of events. Otherwise the event counts for unexpired windows
>>>>     would
>>>>     be 0 which is bad.
>>>     Isn't the latter -- "count would be 0" -- the problem between the
>>>     absence
>>>     of output vs. an output of 0, similar to the use of `Option[T]` in
>>>     Scala
>>>     and the difference between `None` and `Some(0)`?  That is, isn't the
>>>     root
>>>     cause that the downstream system interprets the absence of output in
>>>     a
>>>     particular way ("No output after 1 minute = I consider the output to
>>>     be
>>>     0.")?  Arguably, you could also adapt the downstream system (if
>>>     possible)
>>>     to correctly handle the difference between absence of output vs.
>>>     output of
>>>     0.  I am not implying that we shouldn't care about such a use case,
>>>     but
>>>     want to understand the motivation better. :-)
>>>
>>>     Also, to add some perspective, in some related discussions we talked
>>>     about
>>>     how a Kafka Streams application should not worry or not be coupled
>>>     unnecessarily with such interpretation specifics in a downstream
>>>     system's
>>>     behavior.  After all, tomorrow your app's output might be consumed by
>>>     more
>>>     than just this one downstream system.  Arguably, Kafka Connect rather
>>>     than
>>>     Kafka Streams might be the best tool to link the universes of Kafka
>>>     and
>>>     downstream systems, including helping to reconcile the differences in
>>>     how
>>>     these systems interpret changes, updates, late-arriving data,
>>>     etc.  Kafka
>>>     Connect would allow you to decouple the Kafka Streams app's logical
>>>     processing from the specifics of downstream systems, thanks to
>>>     specific
>>>     sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>>>     this
>>>     decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>>>     but
>>>     it's currently awkward to use Connect for this", this might be a
>>>     problem we
>>>     can solve, too.)
>>>
>>>     Switching topics slightly again.
>>>
>>>     Thomas wrote:
>>>>     I'm not entirely convinced that a separate callback (option C)
>>>>     is that messy (it could just be a default method with an empty
>>>>     implementation), but if we wanted a single API to handle both
>>>>     cases,
>>>>     how about something like the following?
>>>>
>>>>     enum Time {
>>>>         STREAM,
>>>>         CLOCK
>>>>     }
>>>     Yeah, I am on the fence here, too.  If we use the 1-method approach,
>>>     then
>>>     whatever the user is doing inside this method is a black box to Kafka
>>>     Streams (similar to how we have no idea what the user does inside a
>>>     `foreach` -- if the function passed to `foreach` writes to external
>>>     systems, then Kafka Streams is totally unaware of the fact).  We
>>>     won't
>>>     know, for example, if the stream-time action has a smaller "trigger"
>>>     frequency than the processing-time action.  Or, we won't know whether
>>>     the
>>>     user custom-codes a "not later than" trigger logic ("Do X every 1-
>>>     minute of
>>>     stream-time or 1-minute of processing-time, whichever comes
>>>     first").  That
>>>     said, I am not certain yet whether we would need such knowledge
>>>     because,
>>>     when using the Processor API, most of the work and decisions must be
>>>     done
>>>     by the user anyways.  It would matter though if the concept of
>>>     "triggers"
>>>     were to bubble up into the DSL because in the DSL the management of
>>>     windowing, window stores, etc. must be done automatically by Kafka
>>>     Streams.
>>>
>>>     [In any case, btw, we have the corner case where the user configured
>>>     the
>>>     stream-time to be processing-time (e.g. via wall-clock timestamp
>>>     extractor), at which point both punctuate variants are based on the
>>>     same
>>>     time semantics / timeline.]
>>>
>>>     Again, I apologize for the wall of text.  Congratulations if you made
>>>     it
>>>     this far. :-)
>>>
>>>     More than happy to hear your thoughts!
>>>     Michael
>>>
>>>     On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew<ar...@gmail.com> <ma...@gmail.com>
>>>     wrote:
>>>
>>>>     Thanks Matthias.
>>>>     Sure, will correct it right away.
>>>>
>>>>     On 11-Apr-2017 8:07 AM, "Matthias J. Sax"<ma...@confluent.io> <ma...@confluent.io>
>>>>     wrote:
>>>>
>>>>     Thanks for preparing this page!
>>>>
>>>>     About terminology:
>>>>
>>>>     You introduce the term "event time" -- but we should call this
>>>>     "stream
>>>>     time" -- "stream time" is whatever TimestampExtractor returns and
>>>>     this
>>>>     could be event time, ingestion time, or processing/wall-clock time.
>>>>
>>>>     Does this make sense to you?
>>>>
>>>>
>>>>
>>>>     -Matthias
>>>>
>>>>
>>>>     On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>>     Thanks Ewen.
>>>>>
>>>>>     @Michal, @all, I have created a child page to start the Use Cases
>>>>     discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>>>     Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>>     @Tianji, Sorry for the delay. I am trying to make the patch
>>>>>     public.
>>>>>
>>>>>     --
>>>>>     Arun Mathew
>>>>>
>>>>>     On 4/8/17, 02:00, "Ewen Cheslack-Postava"<ew...@confluent.io> <ma...@confluent.io>
>>>>>     wrote:
>>>>>
>>>>>          Arun,
>>>>>
>>>>>          I've given you permission to edit the wiki. Let me know if
>>>>>     you run
>>>>     into any
>>>>>          issues.
>>>>>
>>>>>          -Ewen
>>>>>
>>>>>          On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew<amathew@yahoo-co rp.jp> <ma...@yahoo-corp.jp>
>>>>     wrote:
>>>>>          > Thanks Michal. I dont have the access yet [arunmathew88].
>>>>>     Should I
>>>>     be
>>>>>          > sending a separate mail for this?
>>>>>          >
>>>>>          > I thought one of the person following this thread would be
>>>>>     able to
>>>>     give me
>>>>>          > access.
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > *From: *Michal Borowiecki<mi...@openbet.com>
>>>>>     <ma...@openbet.com>
>>>>>          > *Reply-To: *"dev@kafka.apache.org" <ma...@kafka.apache.org>  <de...@kafka.apache.org> <ma...@kafka.apache.org>
>>>>>          > *Date: *Friday, April 7, 2017 at 17:16
>>>>>          > *To: *"dev@kafka.apache.org" <ma...@kafka.apache.org>  <de...@kafka.apache.org> <ma...@kafka.apache.org>
>>>>>          > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>>>     semantics
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > Hi Arun,
>>>>>          >
>>>>>          > I was thinking along the same lines as you, listing the use
>>>>>     cases
>>>>     on the
>>>>>          > wiki, but didn't find time to get around doing that yet.
>>>>>          > Don't mind if you do it if you have access now.
>>>>>          > I was thinking it would be nice if, once we have the use
>>>>>     cases
>>>>     listed,
>>>>>          > people could use likes to up-vote the use cases similar to
>>>>>     what
>>>>     they're
>>>>>          > working on.
>>>>>          >
>>>>>          > I should have a bit more time to action this in the next
>>>>>     few days,
>>>>     but
>>>>>          > happy for you to do it if you can beat me to it ;-)
>>>>>          >
>>>>>          > Cheers,
>>>>>          > Michal
>>>>>          >
>>>>>          > On 07/04/17 04:39, Arun Mathew wrote:
>>>>>          >
>>>>>          > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > Of course. I was thinking of a subpage where people can
>>>>     collaborate.
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > Will do as per Michaels suggestion.
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > Regards,
>>>>>          >
>>>>>          > Arun Mathew
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > On 4/7/17, 12:30, "Matthias J. Sax"<ma...@confluent.io> <ma...@confluent.io>
>>>>>     <
>>>>     matthias@confluent.io <ma...@confluent.io>> wrote:
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >     Please share your Wiki-ID and a committer can give you
>>>>>     write
>>>>     access.
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >     Btw: as you did not initiate the KIP, you should not
>>>>>     change the
>>>>     KIP
>>>>>          >
>>>>>          >     without the permission of the original author -- in
>>>>>     this case
>>>>     Michael.
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >     So you might also just share your thought over the
>>>>>     mailing list
>>>>     and
>>>>>          >
>>>>>          >     Michael can update the KIP page. Or, as an alternative,
>>>>>     just
>>>>     create a
>>>>>          >
>>>>>          >     subpage for the KIP page.
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >     @Michael: WDYT?
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >     -Matthias
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>>          >
>>>>>          >     > Hi Jay,
>>>>>          >
>>>>>          >     >           Thanks for the advise, I would like to list
>>>>>     down
>>>>     the use cases as
>>>>>          >
>>>>>          >     > per your suggestion. But it seems I don't have write
>>>>     permission to the
>>>>>          >
>>>>>          >     > Apache Kafka Confluent Space. Whom shall I request
>>>>>     for it?
>>>>>          >
>>>>>          >     >
>>>>>          >
>>>>>          >     > Regarding your last question. We are using a patch in
>>>>>     our
>>>>     production system
>>>>>          >
>>>>>          >     > which does exactly this.
>>>>>          >
>>>>>          >     > We window by the event time, but trigger punctuate in
>>>>     <punctuate interval>
>>>>>          >
>>>>>          >     > duration of system time, in the absence of an event
>>>>>     crossing
>>>>     the punctuate
>>>>>          >
>>>>>          >     > event time.
>>>>>          >
>>>>>          >     >
>>>>>          >
>>>>>          >     > We are using Kafka Stream for our Audit Trail, where
>>>>>     we need
>>>>     to output the
>>>>>          >
>>>>>          >     > event counts on each topic on each cluster aggregated
>>>>>     over a
>>>>     1 minute
>>>>>          >
>>>>>          >     > window. We have to use event time to be able to cross
>>>>>     check
>>>>     the counts. But
>>>>>          >
>>>>>          >     > we need to trigger punctuate [aggregate event pushes]
>>>>>     by
>>>>     system time in the
>>>>>          >
>>>>>          >     > absence of events. Otherwise the event counts for
>>>>>     unexpired
>>>>     windows would
>>>>>          >
>>>>>          >     > be 0 which is bad.
>>>>>          >
>>>>>          >     >
>>>>>          >
>>>>>          >     > "Maybe a hybrid solution works: I window by event
>>>>>     time but
>>>>     trigger results
>>>>>          >
>>>>>          >     > by system time for windows that have updated? Not
>>>>>     really sure
>>>>     the details
>>>>>          >
>>>>>          >     > of making that work. Does that work? Are there
>>>>>     concrete
>>>>     examples where you
>>>>>          >
>>>>>          >     > actually want the current behavior?"
>>>>>          >
>>>>>          >     >
>>>>>          >
>>>>>          >     > --
>>>>>          >
>>>>>          >     > With Regards,
>>>>>          >
>>>>>          >     >
>>>>>          >
>>>>>          >     > Arun Mathew
>>>>>          >
>>>>>          >     > Yahoo! JAPAN Corporation
>>>>>          >
>>>>>          >     >
>>>>>          >
>>>>>          >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>>>     skyahead@gmail.com <ma...@gmail.com>>
>>>>     <sk...@gmail.com> <ma...@gmail.com>  wrote:
>>>>>          >
>>>>>          >     >
>>>>>          >
>>>>>          >     >> Hi Jay,
>>>>>          >
>>>>>          >     >>
>>>>>          >
>>>>>          >     >> The hybrid solution is exactly what I expect and
>>>>>     need for
>>>>     our use cases
>>>>>          >
>>>>>          >     >> when dealing with telecom data.
>>>>>          >
>>>>>          >     >>
>>>>>          >
>>>>>          >     >> Thanks
>>>>>          >
>>>>>          >     >> Tianji
>>>>>          >
>>>>>          >     >>
>>>>>          >
>>>>>          >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>>>     jay@confluent.io <ma...@confluent.io>>
>>>>     <ja...@confluent.io> <ma...@confluent.io>  wrote:
>>>>>          >
>>>>>          >     >>
>>>>>          >
>>>>>          >     >>> Hey guys,
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>> One thing I've always found super important for
>>>>>     this kind
>>>>     of design work
>>>>>          >
>>>>>          >     >> is
>>>>>          >
>>>>>          >     >>> to do a really good job of cataloging the landscape
>>>>>     of use
>>>>     cases and how
>>>>>          >
>>>>>          >     >>> prevalent each one is. By that I mean not just
>>>>>     listing lots
>>>>     of uses, but
>>>>>          >
>>>>>          >     >>> also grouping them into categories that
>>>>>     functionally need
>>>>     the same thing.
>>>>>          >
>>>>>          >     >>> In the absence of this it is very hard to reason
>>>>>     about
>>>>     design proposals.
>>>>>          >
>>>>>          >     >>> From the proposals so far I think we have a lot of
>>>>     discussion around
>>>>>          >
>>>>>          >     >>> possible apis, but less around what the user needs
>>>>>     for
>>>>     different use
>>>>>          >
>>>>>          >     >> cases
>>>>>          >
>>>>>          >     >>> and how they would implement that using the api.
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>> Here is an example:
>>>>>          >
>>>>>          >     >>> You aggregate click and impression data for a
>>>>>     reddit like
>>>>     site. Every ten
>>>>>          >
>>>>>          >     >>> minutes you want to output a ranked list of the top
>>>>>     10
>>>>     articles ranked by
>>>>>          >
>>>>>          >     >>> clicks/impressions for each geographical area. I
>>>>>     want to be
>>>>     able run this
>>>>>          >
>>>>>          >     >>> in steady state as well as rerun to regenerate
>>>>>     results (or
>>>>     catch up if it
>>>>>          >
>>>>>          >     >>> crashes).
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>> There are a couple of tricky things that seem to
>>>>>     make this
>>>>     hard with
>>>>>          >
>>>>>          >     >> either
>>>>>          >
>>>>>          >     >>> of the options proposed:
>>>>>          >
>>>>>          >     >>> 1. If I emit this data using event time I have the
>>>>>     problem
>>>>     described
>>>>>          >
>>>>>          >     >> where
>>>>>          >
>>>>>          >     >>> a geographical region with no new clicks or
>>>>>     impressions
>>>>     will fail to
>>>>>          >
>>>>>          >     >> output
>>>>>          >
>>>>>          >     >>> results.
>>>>>          >
>>>>>          >     >>> 2. If I emit this data using system time I have the
>>>>>     problem
>>>>     that when
>>>>>          >
>>>>>          >     >>> reprocessing data my window may not be ten minutes
>>>>>     but 10
>>>>     hours if my
>>>>>          >
>>>>>          >     >>> processing is very fast so it dramatically changes
>>>>>     the
>>>>     output.
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>> Maybe a hybrid solution works: I window by event
>>>>>     time but
>>>>     trigger results
>>>>>          >
>>>>>          >     >>> by system time for windows that have updated? Not
>>>>>     really
>>>>     sure the details
>>>>>          >
>>>>>          >     >>> of making that work. Does that work? Are there
>>>>>     concrete
>>>>     examples where
>>>>>          >
>>>>>          >     >> you
>>>>>          >
>>>>>          >     >>> actually want the current behavior?
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>> -Jay
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>>>     arunmathew88@gmail.com <ma...@gmail.com> <ma...@gmail.com>
>>>>>          >
>>>>>          >     >>> wrote:
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>>> Hi All,
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> Thanks for the KIP. We were also in need of a
>>>>>     mechanism to
>>>>     trigger
>>>>>          >
>>>>>          >     >>>> punctuate in the absence of events.
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> As I described in [
>>>>>          >
>>>>>          >     >>>>https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>>          >
>>>>>          >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>>          >
>>>>>          >     >>>> plugin.system.issuetabpanels:comment-
>>>>>     tabpanel#comment-
>>>>     15926036
>>>>>          >
>>>>>          >     >>>> ],
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>>    - Our approached involved using the event time
>>>>>     by
>>>>     default.
>>>>>          >
>>>>>          >     >>>>    - The method to check if there is any punctuate
>>>>>     ready
>>>>     in the
>>>>>          >
>>>>>          >     >>>>    PunctuationQueue is triggered via the any event
>>>>     received by the
>>>>>          >
>>>>>          >     >> stream
>>>>>          >
>>>>>          >     >>>>    tread, or at the polling intervals in the
>>>>>     absence of
>>>>     any events.
>>>>>          >
>>>>>          >     >>>>    - When we create Punctuate objects (which
>>>>>     contains the
>>>>     next event
>>>>>          >
>>>>>          >     >> time
>>>>>          >
>>>>>          >     >>>>    for punctuation and interval), we also record
>>>>>     the
>>>>     creation time
>>>>>          >
>>>>>          >     >>> (system
>>>>>          >
>>>>>          >     >>>>    time).
>>>>>          >
>>>>>          >     >>>>    - While checking for maturity of Punctuate
>>>>>     Schedule by
>>>>>          >
>>>>>          >     >> mayBePunctuate
>>>>>          >
>>>>>          >     >>>>    method, we also check if the system clock has
>>>>>     elapsed
>>>>     the punctuate
>>>>>          >
>>>>>          >     >>>>    interval since the schedule creation time.
>>>>>          >
>>>>>          >     >>>>    - In the absence of any event, or in the
>>>>>     absence of any
>>>>     event for
>>>>>          >
>>>>>          >     >> one
>>>>>          >
>>>>>          >     >>>>    topic in the partition group assigned to the
>>>>>     stream
>>>>     task, the system
>>>>>          >
>>>>>          >     >>>> time
>>>>>          >
>>>>>          >     >>>>    will elapse the interval and we trigger a
>>>>>     punctuate
>>>>     using the
>>>>>          >
>>>>>          >     >> expected
>>>>>          >
>>>>>          >     >>>>    punctuation event time.
>>>>>          >
>>>>>          >     >>>>    - we then create the next punctuation schedule
>>>>>     as
>>>>     punctuation event
>>>>>          >
>>>>>          >     >>> time
>>>>>          >
>>>>>          >     >>>>    + punctuation interval, [again recording the
>>>>>     system
>>>>     time of creation
>>>>>          >
>>>>>          >     >>> of
>>>>>          >
>>>>>          >     >>>> the
>>>>>          >
>>>>>          >     >>>>    schedule].
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>>>     approach
>>>>     has pros and
>>>>>          >
>>>>>          >     >>>> cons.
>>>>>          >
>>>>>          >     >>>> Pros
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>>    - Punctuates will happen in <punctuate
>>>>>     interval> time
>>>>     duration at
>>>>>          >
>>>>>          >     >> max
>>>>>          >
>>>>>          >     >>> in
>>>>>          >
>>>>>          >     >>>>    terms of system time.
>>>>>          >
>>>>>          >     >>>>    - The semantics as a whole continues to revolve
>>>>>     around
>>>>     event time.
>>>>>          >
>>>>>          >     >>>>    - We can use the old data [old timestamps] to
>>>>>     rerun any
>>>>     experiments
>>>>>          >
>>>>>          >     >> or
>>>>>          >
>>>>>          >     >>>>    tests.
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> Cons
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>>    - In case the  <punctuate interval> is not a
>>>>>     time
>>>>     duration [say
>>>>>          >
>>>>>          >     >>> logical
>>>>>          >
>>>>>          >     >>>>    time/event count], then the approach might not
>>>>>     be
>>>>     meaningful.
>>>>>          >
>>>>>          >     >>>>    - In case there is a case where we have to wait
>>>>>     for an
>>>>     actual event
>>>>>          >
>>>>>          >     >>> from
>>>>>          >
>>>>>          >     >>>>    a low event rate partition in the partition
>>>>>     group, this
>>>>     approach
>>>>>          >
>>>>>          >     >> will
>>>>>          >
>>>>>          >     >>>> jump
>>>>>          >
>>>>>          >     >>>>    the gun.
>>>>>          >
>>>>>          >     >>>>    - in case the event processing cannot catch up
>>>>>     with the
>>>>     event rate
>>>>>          >
>>>>>          >     >> and
>>>>>          >
>>>>>          >     >>>>    the expected timestamp events gets queued for
>>>>>     long
>>>>     time, this
>>>>>          >
>>>>>          >     >> approach
>>>>>          >
>>>>>          >     >>>>    might jump the gun.
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> I believe the above approach and discussion goes
>>>>>     close to
>>>>     the approach
>>>>>          >
>>>>>          >     >> A.
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> -----------
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> I like the idea of having an even count based
>>>>>     punctuate.
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> -----------
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> I agree with the discussion around approach C,
>>>>>     that we
>>>>     should provide
>>>>>          >
>>>>>          >     >> the
>>>>>          >
>>>>>          >     >>>> user with the option to choose system time or
>>>>>     event time
>>>>     based
>>>>>          >
>>>>>          >     >>> punctuates.
>>>>>          >
>>>>>          >     >>>> But I believe that the user predominantly wants to
>>>>>     use
>>>>     event time while
>>>>>          >
>>>>>          >     >>> not
>>>>>          >
>>>>>          >     >>>> missing out on regular punctuates due to event
>>>>>     delays or
>>>>     event
>>>>>          >
>>>>>          >     >> absences.
>>>>>          >
>>>>>          >     >>>> Hence a complex punctuate option as Matthias
>>>>>     mentioned
>>>>     (quoted below)
>>>>>          >
>>>>>          >     >>> would
>>>>>          >
>>>>>          >     >>>> be most apt.
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> "- We might want to add "complex" schedules later
>>>>>     on
>>>>     (like, punctuate
>>>>>          >
>>>>>          >     >> on
>>>>>          >
>>>>>          >     >>>> every 10 seconds event-time or 60 seconds system-
>>>>>     time
>>>>     whatever comes
>>>>>          >
>>>>>          >     >>>> first)."
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> -----------
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> I think I read somewhere that Kafka Streams
>>>>>     started with
>>>>     System Time as
>>>>>          >
>>>>>          >     >>> the
>>>>>          >
>>>>>          >     >>>> punctuation standard, but was later changed to
>>>>>     Event Time.
>>>>     I guess
>>>>>          >
>>>>>          >     >> there
>>>>>          >
>>>>>          >     >>>> would be some good reason behind it. As Kafka
>>>>>     Streams want
>>>>     to evolve
>>>>>          >
>>>>>          >     >> more
>>>>>          >
>>>>>          >     >>>> on the Stream Processing front, I believe the
>>>>>     emphasis on
>>>>     event time
>>>>>          >
>>>>>          >     >>> would
>>>>>          >
>>>>>          >     >>>> remain quite strong.
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> With Regards,
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> Arun Mathew
>>>>>          >
>>>>>          >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>>>     tobecker@tivo.com <ma...@tivo.com> <ma...@tivo.com>
>>>>>          >
>>>>>          >     >> wrote:
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>>> Yeah I like PuncutationType much better; I just
>>>>>     threw
>>>>     Time out there
>>>>>          >
>>>>>          >     >>>>> more as a strawman than an actual suggestion ;) I
>>>>>     still
>>>>     think it's
>>>>>          >
>>>>>          >     >>>>> worth considering what this buys us over an
>>>>>     additional
>>>>     callback. I
>>>>>          >
>>>>>          >     >>>>> foresee a number of punctuate implementations
>>>>>     following
>>>>     this pattern:
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>> public void punctuate(PunctuationType type) {
>>>>>          >
>>>>>          >     >>>>>     switch (type) {
>>>>>          >
>>>>>          >     >>>>>         case EVENT_TIME:
>>>>>          >
>>>>>          >     >>>>>             methodA();
>>>>>          >
>>>>>          >     >>>>>             break;
>>>>>          >
>>>>>          >     >>>>>         case SYSTEM_TIME:
>>>>>          >
>>>>>          >     >>>>>             methodB();
>>>>>          >
>>>>>          >     >>>>>             break;
>>>>>          >
>>>>>          >     >>>>>     }
>>>>>          >
>>>>>          >     >>>>> }
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>> I guess one advantage of this approach is we
>>>>>     could add
>>>>     additional
>>>>>          >
>>>>>          >     >>>>> punctuation types later in a backwards compatible
>>>>>     way
>>>>     (like event
>>>>>          >
>>>>>          >     >> count
>>>>>          >
>>>>>          >     >>>>> as you mentioned).
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>> -Tommy
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>>>     Sax wrote:
>>>>>          >
>>>>>          >     >>>>>> That sounds promising.
>>>>>          >
>>>>>          >     >>>>>>
>>>>>          >
>>>>>          >     >>>>>> I am just wondering if `Time` is the best name.
>>>>>     Maybe we
>>>>     want to
>>>>>          >
>>>>>          >     >> add
>>>>>          >
>>>>>          >     >>>>>> other non-time based punctuations at some point
>>>>>     later. I
>>>>     would
>>>>>          >
>>>>>          >     >>>>>> suggest
>>>>>          >
>>>>>          >     >>>>>>
>>>>>          >
>>>>>          >     >>>>>> enum PunctuationType {
>>>>>          >
>>>>>          >     >>>>>>   EVENT_TIME,
>>>>>          >
>>>>>          >     >>>>>>   SYSTEM_TIME,
>>>>>          >
>>>>>          >     >>>>>> }
>>>>>          >
>>>>>          >     >>>>>>
>>>>>          >
>>>>>          >     >>>>>> or similar. Just to keep the door open -- it's
>>>>>     easier to
>>>>     add new
>>>>>          >
>>>>>          >     >>>>>> stuff
>>>>>          >
>>>>>          >     >>>>>> if the name is more generic.
>>>>>          >
>>>>>          >     >>>>>>
>>>>>          >
>>>>>          >     >>>>>>
>>>>>          >
>>>>>          >     >>>>>> -Matthias
>>>>>          >
>>>>>          >     >>>>>>
>>>>>          >
>>>>>          >     >>>>>>
>>>>>          >
>>>>>          >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>> I agree that the framework providing and
>>>>>     managing the
>>>>     notion of
>>>>>          >
>>>>>          >     >>>>>>> stream
>>>>>          >
>>>>>          >     >>>>>>> time is valuable and not something we would
>>>>>     want to
>>>>     delegate to
>>>>>          >
>>>>>          >     >> the
>>>>>          >
>>>>>          >     >>>>>>> tasks. I'm not entirely convinced that a
>>>>>     separate
>>>>     callback
>>>>>          >
>>>>>          >     >> (option
>>>>>          >
>>>>>          >     >>>>>>> C)
>>>>>          >
>>>>>          >     >>>>>>> is that messy (it could just be a default
>>>>>     method with
>>>>     an empty
>>>>>          >
>>>>>          >     >>>>>>> implementation), but if we wanted a single API
>>>>>     to
>>>>     handle both
>>>>>          >
>>>>>          >     >>>>>>> cases,
>>>>>          >
>>>>>          >     >>>>>>> how about something like the following?
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>> enum Time {
>>>>>          >
>>>>>          >     >>>>>>>    STREAM,
>>>>>          >
>>>>>          >     >>>>>>>    CLOCK
>>>>>          >
>>>>>          >     >>>>>>> }
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>> Then on ProcessorContext:
>>>>>          >
>>>>>          >     >>>>>>> context.schedule(Time time, long interval)  //
>>>>>     We could
>>>>     allow
>>>>>          >
>>>>>          >     >> this
>>>>>          >
>>>>>          >     >>>>>>> to
>>>>>          >
>>>>>          >     >>>>>>> be called once for each value of time to mix
>>>>     approaches.
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>> Then the Processor API becomes:
>>>>>          >
>>>>>          >     >>>>>>> punctuate(Time time) // time here denotes which
>>>>     schedule resulted
>>>>>          >
>>>>>          >     >>>>>>> in
>>>>>          >
>>>>>          >     >>>>>>> this call.
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>> Thoughts?
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>>>     Sax
>>>>     wrote:
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> I was thinking about the four options you
>>>>>     proposed in
>>>>     more
>>>>>          >
>>>>>          >     >>>>>>>> details
>>>>>          >
>>>>>          >     >>>>>>>> and
>>>>>          >
>>>>>          >     >>>>>>>> this are my thoughts:
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> (A) You argue, that users can still
>>>>>     "punctuate" on
>>>>     event-time
>>>>>          >
>>>>>          >     >> via
>>>>>          >
>>>>>          >     >>>>>>>> process(), but I am not sure if this is
>>>>>     possible.
>>>>     Note, that
>>>>>          >
>>>>>          >     >>>>>>>> users
>>>>>          >
>>>>>          >     >>>>>>>> only
>>>>>          >
>>>>>          >     >>>>>>>> get record timestamps via context.timestamp().
>>>>>     Thus,
>>>>     users
>>>>>          >
>>>>>          >     >> would
>>>>>          >
>>>>>          >     >>>>>>>> need
>>>>>          >
>>>>>          >     >>>>>>>> to
>>>>>          >
>>>>>          >     >>>>>>>> track the time progress per partition (based
>>>>>     on the
>>>>     partitions
>>>>>          >
>>>>>          >     >>>>>>>> they
>>>>>          >
>>>>>          >     >>>>>>>> obverse via context.partition(). (This alone
>>>>>     puts a
>>>>     huge burden
>>>>>          >
>>>>>          >     >>>>>>>> on
>>>>>          >
>>>>>          >     >>>>>>>> the
>>>>>          >
>>>>>          >     >>>>>>>> user by itself.) However, users are not
>>>>>     notified at
>>>>     startup
>>>>>          >
>>>>>          >     >> what
>>>>>          >
>>>>>          >     >>>>>>>> partitions are assigned, and user are not
>>>>>     notified
>>>>     when
>>>>>          >
>>>>>          >     >>>>>>>> partitions
>>>>>          >
>>>>>          >     >>>>>>>> get
>>>>>          >
>>>>>          >     >>>>>>>> revoked. Because this information is not
>>>>>     available,
>>>>     it's not
>>>>>          >
>>>>>          >     >>>>>>>> possible
>>>>>          >
>>>>>          >     >>>>>>>> to
>>>>>          >
>>>>>          >     >>>>>>>> "manually advance" stream-time, and thus
>>>>>     event-time
>>>>     punctuation
>>>>>          >
>>>>>          >     >>>>>>>> within
>>>>>          >
>>>>>          >     >>>>>>>> process() seems not to be possible -- or do
>>>>>     you see a
>>>>     way to
>>>>>          >
>>>>>          >     >> get
>>>>>          >
>>>>>          >     >>>>>>>> it
>>>>>          >
>>>>>          >     >>>>>>>> done? And even if, it might still be too
>>>>>     clumsy to
>>>>     use.
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> (B) This does not allow to mix both
>>>>>     approaches, thus
>>>>     limiting
>>>>>          >
>>>>>          >     >>>>>>>> what
>>>>>          >
>>>>>          >     >>>>>>>> users
>>>>>          >
>>>>>          >     >>>>>>>> can do.
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> (C) This should give all flexibility we need.
>>>>>     However,
>>>>     just
>>>>>          >
>>>>>          >     >>>>>>>> adding
>>>>>          >
>>>>>          >     >>>>>>>> one
>>>>>          >
>>>>>          >     >>>>>>>> more method seems to be a solution that is too
>>>>>     simple
>>>>     (cf my
>>>>>          >
>>>>>          >     >>>>>>>> comments
>>>>>          >
>>>>>          >     >>>>>>>> below).
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>>>     sure how
>>>>     a user
>>>>>          >
>>>>>          >     >>>>>>>> could
>>>>>          >
>>>>>          >     >>>>>>>> enable system-time and event-time punctuation
>>>>>     in
>>>>     parallel.
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> Overall options (C) seems to be the most
>>>>>     promising
>>>>     approach to
>>>>>          >
>>>>>          >     >>>>>>>> me.
>>>>>          >
>>>>>          >     >>>>>>>> Because I also favor a clean API, we might
>>>>>     keep
>>>>     current
>>>>>          >
>>>>>          >     >>>>>>>> punctuate()
>>>>>          >
>>>>>          >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>>>     at some
>>>>     later
>>>>>          >
>>>>>          >     >>>>>>>> point
>>>>>          >
>>>>>          >     >>>>>>>> when
>>>>>          >
>>>>>          >     >>>>>>>> people use the "new punctuate API".
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> Couple of follow up questions:
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> - I am wondering, if we should have two
>>>>>     callback
>>>>     methods or
>>>>>          >
>>>>>          >     >> just
>>>>>          >
>>>>>          >     >>>>>>>> one
>>>>>          >
>>>>>          >     >>>>>>>> (ie, a unified for system and event time
>>>>>     punctuation
>>>>     or one for
>>>>>          >
>>>>>          >     >>>>>>>> each?).
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> - If we have one, how can the user figure out,
>>>>>     which
>>>>     condition
>>>>>          >
>>>>>          >     >>>>>>>> did
>>>>>          >
>>>>>          >     >>>>>>>> trigger?
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> - How would the API look like, for registering
>>>>     different
>>>>>          >
>>>>>          >     >>>>>>>> punctuate
>>>>>          >
>>>>>          >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> - We might want to add "complex" schedules
>>>>>     later on
>>>>     (like,
>>>>>          >
>>>>>          >     >>>>>>>> punctuate
>>>>>          >
>>>>>          >     >>>>>>>> on
>>>>>          >
>>>>>          >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>>>     system-time
>>>>     whatever
>>>>>          >
>>>>>          >     >>>>>>>> comes
>>>>>          >
>>>>>          >     >>>>>>>> first). I don't say we should add this right
>>>>>     away, but
>>>>     we might
>>>>>          >
>>>>>          >     >>>>>>>> want
>>>>>          >
>>>>>          >     >>>>>>>> to
>>>>>          >
>>>>>          >     >>>>>>>> define the API in a way, that it allows
>>>>>     extensions
>>>>     like this
>>>>>          >
>>>>>          >     >>>>>>>> later
>>>>>          >
>>>>>          >     >>>>>>>> on,
>>>>>          >
>>>>>          >     >>>>>>>> without redesigning the API (ie, the API
>>>>>     should be
>>>>     designed
>>>>>          >
>>>>>          >     >>>>>>>> extensible)
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> - Did you ever consider count-based
>>>>>     punctuation?
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> I understand, that you would like to solve a
>>>>>     simple
>>>>     problem,
>>>>>          >
>>>>>          >     >> but
>>>>>          >
>>>>>          >     >>>>>>>> we
>>>>>          >
>>>>>          >     >>>>>>>> learned from the past, that just "adding some
>>>>>     API"
>>>>     quickly
>>>>>          >
>>>>>          >     >> leads
>>>>>          >
>>>>>          >     >>>>>>>> to a
>>>>>          >
>>>>>          >     >>>>>>>> not very well defined API that needs time
>>>>>     consuming
>>>>     clean up
>>>>>          >
>>>>>          >     >>>>>>>> later on
>>>>>          >
>>>>>          >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>>>     holistic
>>>>>          >
>>>>>          >     >>>>>>>> punctuation
>>>>>          >
>>>>>          >     >>>>>>>> KIP
>>>>>          >
>>>>>          >     >>>>>>>> with this from the beginning on to avoid later
>>>>>     painful
>>>>>          >
>>>>>          >     >> redesign.
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> -Matthias
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>> Thanks Thomas,
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>> I'm also wary of changing the existing
>>>>>     semantics of
>>>>>          >
>>>>>          >     >> punctuate,
>>>>>          >
>>>>>          >     >>>>>>>>> for
>>>>>          >
>>>>>          >     >>>>>>>>> backward compatibility reasons, although I
>>>>>     like the
>>>>>          >
>>>>>          >     >> conceptual
>>>>>          >
>>>>>          >     >>>>>>>>> simplicity of that option.
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>>>     a way,
>>>>     uglier.
>>>>>          >
>>>>>          >     >> I
>>>>>          >
>>>>>          >     >>>>>>>>> added
>>>>>          >
>>>>>          >     >>>>>>>>> this to the KIP now as option (C).
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>>>     more
>>>>     flexible,
>>>>>          >
>>>>>          >     >> as
>>>>>          >
>>>>>          >     >>>>>>>>> it
>>>>>          >
>>>>>          >     >>>>>>>>> allows
>>>>>          >
>>>>>          >     >>>>>>>>> you to return any value, you're not limited
>>>>>     to event
>>>>     time or
>>>>>          >
>>>>>          >     >>>>>>>>> system
>>>>>          >
>>>>>          >     >>>>>>>>> time
>>>>>          >
>>>>>          >     >>>>>>>>> (although I don't see an actual use case
>>>>>     where you
>>>>     might need
>>>>>          >
>>>>>          >     >>>>>>>>> anything
>>>>>          >
>>>>>          >     >>>>>>>>> else then those two). Hence I also proposed
>>>>>     the
>>>>     option to
>>>>>          >
>>>>>          >     >> allow
>>>>>          >
>>>>>          >     >>>>>>>>> users
>>>>>          >
>>>>>          >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>>>     for
>>>>     them given
>>>>>          >
>>>>>          >     >>>>>>>>> the
>>>>>          >
>>>>>          >     >>>>>>>>> presence or absence of messages, much like
>>>>>     they can
>>>>     decide
>>>>>          >
>>>>>          >     >> what
>>>>>          >
>>>>>          >     >>>>>>>>> msg
>>>>>          >
>>>>>          >     >>>>>>>>> time
>>>>>          >
>>>>>          >     >>>>>>>>> means for them using the TimestampExtractor.
>>>>>     What do
>>>>     you
>>>>>          >
>>>>>          >     >> think
>>>>>          >
>>>>>          >     >>>>>>>>> about
>>>>>          >
>>>>>          >     >>>>>>>>> that? This is probably most flexible but also
>>>>>     most
>>>>>          >
>>>>>          >     >> complicated.
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>> All comments appreciated.
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>> Cheers,
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>> Michal
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>          >
>>>>>          >     >>>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>> Although I fully agree we need a way to
>>>>>     trigger
>>>>     periodic
>>>>>          >
>>>>>          >     >>>>>>>>>> processing
>>>>>          >
>>>>>          >     >>>>>>>>>> that is independent from whether and when
>>>>>     messages
>>>>     arrive,
>>>>>          >
>>>>>          >     >>>>>>>>>> I'm
>>>>>          >
>>>>>          >     >>>>>>>>>> not sure
>>>>>          >
>>>>>          >     >>>>>>>>>> I like the idea of changing the existing
>>>>>     semantics
>>>>     across
>>>>>          >
>>>>>          >     >> the
>>>>>          >
>>>>>          >     >>>>>>>>>> board.
>>>>>          >
>>>>>          >     >>>>>>>>>> What if we added an additional callback to
>>>>>     Processor
>>>>     that
>>>>>          >
>>>>>          >     >> can
>>>>>          >
>>>>>          >     >>>>>>>>>> be
>>>>>          >
>>>>>          >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>>>     always
>>>>     called at
>>>>>          >
>>>>>          >     >>>>>>>>>> fixed, wall
>>>>>          >
>>>>>          >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>>>     have to
>>>>     give
>>>>>          >
>>>>>          >     >> up
>>>>>          >
>>>>>          >     >>>>>>>>>> the
>>>>>          >
>>>>>          >     >>>>>>>>>> notion
>>>>>          >
>>>>>          >     >>>>>>>>>> of stream time to be able to do periodic
>>>>>     processing.
>>>>>          >
>>>>>          >     >>>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>>>     Borowiecki
>>>>     wrote:
>>>>>          >
>>>>>          >     >>>>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>>> Hi all,
>>>>>          >
>>>>>          >     >>>>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>>>     punctuate
>>>>>          >
>>>>>          >     >>>>>>>>>>> semantics
>>>>>          >
>>>>>          >     >>>>>>>>>>> <https://cwiki.apache.org/
>>>>     confluence/display/KAFKA/KIP-<https://cwiki.apache.org/ confluence/display/KAFKA/KIP->
>>>>     <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>>>>>          >
>>>>>          >     ><https://cwiki.apache.org/confluence/display/KAFKA/KI P->
>>>>>     <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>>>>     138%
>>>>>          >
>>>>>          >     >>>>>>>>>>> 3A+C
>>>>>          >
>>>>>          >     >>>>>>>>>>> hange+
>>>>>          >
>>>>>          >     >>>>>>>>>>> punctuate+semantics>
>>>>>          >
>>>>>          >     >>>>>>>>>>> .
>>>>>          >
>>>>>          >     >>>>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>>> Appreciating there can be different views
>>>>>     on
>>>>     system-time
>>>>>          >
>>>>>          >     >> vs
>>>>>          >
>>>>>          >     >>>>>>>>>>> event-
>>>>>          >
>>>>>          >     >>>>>>>>>>> time
>>>>>          >
>>>>>          >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>>>     case and
>>>>     the
>>>>>          >
>>>>>          >     >>>>>>>>>>> importance of
>>>>>          >
>>>>>          >     >>>>>>>>>>> backwards compatibility of any such change,
>>>>>     I've
>>>>     left it
>>>>>          >
>>>>>          >     >>>>>>>>>>> quite
>>>>>          >
>>>>>          >     >>>>>>>>>>> open
>>>>>          >
>>>>>          >     >>>>>>>>>>> and
>>>>>          >
>>>>>          >     >>>>>>>>>>> hope to fill in more info as the discussion
>>>>     progresses.
>>>>>          >
>>>>>          >     >>>>>>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>>>>> Thanks,
>>>>>          >
>>>>>          >     >>>>>>>>>>> Michal
>>>>>          >
>>>>>          >     >>>>>>> --
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>     Tommy Becker
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>     Senior Software Engineer
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>     O+1 919.460.4747 <tel:%28919%29%20460-4747>  <(919)%20460-4747>
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>tivo.com <http://tivo.com>
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>> ________________________________
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>>>> This email and any attachments may contain
>>>>>     confidential
>>>>     and
>>>>>          >
>>>>>          >     >>>>>>> privileged material for the sole use of the
>>>>>     intended
>>>>     recipient.
>>>>>          >
>>>>>          >     >> Any
>>>>>          >
>>>>>          >     >>>>>>> review, copying, or distribution of this email
>>>>>     (or any
>>>>>          >
>>>>>          >     >> attachments)
>>>>>          >
>>>>>          >     >>>>>>> by others is prohibited. If you are not the
>>>>>     intended
>>>>     recipient,
>>>>>          >
>>>>>          >     >>>>>>> please contact the sender immediately and
>>>>>     permanently
>>>>     delete this
>>>>>          >
>>>>>          >     >>>>>>> email and any attachments. No employee or agent
>>>>>     of TiVo
>>>>     Inc. is
>>>>>          >
>>>>>          >     >>>>>>> authorized to conclude any binding agreement on
>>>>>     behalf
>>>>     of TiVo
>>>>>          >
>>>>>          >     >> Inc.
>>>>>          >
>>>>>          >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>>>     only be
>>>>     made by a
>>>>>          >
>>>>>          >     >>>>>>> signed written agreement.
>>>>>          >
>>>>>          >     >>>>>>>
>>>>>          >
>>>>>          >     >>>>> --
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>>     Tommy Becker
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>>     Senior Software Engineer
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>>     O+1 919.460.4747 <tel:%28919%29%20460-4747>  <(919)%20460-4747>
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>>tivo.com <http://tivo.com>
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>> ________________________________
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>> This email and any attachments may contain
>>>>>     confidential
>>>>     and
>>>>>          >
>>>>>          >     >> privileged
>>>>>          >
>>>>>          >     >>>>> material for the sole use of the intended
>>>>>     recipient. Any
>>>>     review,
>>>>>          >
>>>>>          >     >>> copying,
>>>>>          >
>>>>>          >     >>>>> or distribution of this email (or any
>>>>>     attachments) by
>>>>     others is
>>>>>          >
>>>>>          >     >>>> prohibited.
>>>>>          >
>>>>>          >     >>>>> If you are not the intended recipient, please
>>>>>     contact the
>>>>     sender
>>>>>          >
>>>>>          >     >>>>> immediately and permanently delete this email and
>>>>>     any
>>>>     attachments. No
>>>>>          >
>>>>>          >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>>>     conclude
>>>>     any binding
>>>>>          >
>>>>>          >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>>>     Binding
>>>>     agreements with
>>>>>          >
>>>>>          >     >> TiVo
>>>>>          >
>>>>>          >     >>>>> Inc. may only be made by a signed written
>>>>>     agreement.
>>>>>          >
>>>>>          >     >>>>>
>>>>>          >
>>>>>          >     >>>>
>>>>>          >
>>>>>          >     >>>
>>>>>          >
>>>>>          >     >>
>>>>>          >
>>>>>          >     >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > --
>>>>>          >
>>>>>          ><http://www.openbet.com/> <http://www.openbet.com/>
>>>>>          >
>>>>>          > *Michal Borowiecki*
>>>>>          >
>>>>>          > *Senior Software Engineer L4*
>>>>>          >
>>>>>          > *T: *
>>>>>          >
>>>>>          >+44 208 742 1600 <tel:+44%2020%208742%201600>  <+44%2020%208742%201600>
>>>>>          >
>>>>>          >+44 203 249 8448 <tel:+44%2020%203249%208448>  <+44%2020%203249%208448>
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > *E: *
>>>>>          >
>>>>>          >michal.borowiecki@openbet.com
>>>>>     <ma...@openbet.com>
>>>>>          >
>>>>>          > *W: *
>>>>>          >
>>>>>          >www.openbet.com <http://www.openbet.com>
>>>>>          >
>>>>>          > *OpenBet Ltd*
>>>>>          >
>>>>>          > Chiswick Park Building 9
>>>>>          >
>>>>>          > 566 Chiswick High Rd
>>>>>          >
>>>>>          > London
>>>>>          >
>>>>>          > W4 5XT
>>>>>          >
>>>>>          > UK
>>>>>          >
>>>>>          ><https://www.openbet.com/email_promo>
>>>>>     <https://www.openbet.com/email_promo>
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>          > This message is confidential and intended only for the
>>>>>     addressee.
>>>>     If you
>>>>>          > have received this message in error, please immediately
>>>>>     notify the
>>>>>          >postmaster@openbet.com <ma...@openbet.com>  and delete it from your system as
>>>>>     well as
>>>>     any
>>>>>          > copies. The content of e-mails as well as traffic data may
>>>>>     be
>>>>     monitored by
>>>>>          > OpenBet for employment and security purposes. To protect
>>>>>     the
>>>>     environment
>>>>>          > please do not print this e-mail unless necessary. OpenBet
>>>>>     Ltd.
>>>>     Registered
>>>>>          > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>>>     London,
>>>>     W4 5XT,
>>>>>          > United Kingdom. A company registered in England and Wales.
>>>>     Registered no.
>>>>>          > 3134634. VAT no. GB927523612
>>>>>          >
>>>>>          >
>>>>>          >
>>>>>
>>>>>
>>     --
>>
>>
>>          Tommy Becker
>>
>>          Senior Software Engineer
>>
>>          O+1 919.460.4747 <tel:%28919%29%20460-4747>
>>          tivo.com <http://tivo.com>
>>
>>     ________________________________
>>
>>     This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>
>     -- 
>     <http://www.openbet.com/> 	Michal Borowiecki
>     Senior Software Engineer L4
>     	T: 	+44 208 742 1600 <tel:+44%2020%208742%201600>
>
>     	
>     	+44 203 249 8448 <tel:+44%2020%203249%208448>
>
>     	
>     	
>     	E: 	michal.borowiecki@openbet.com
>     <ma...@openbet.com>
>     	W: 	www.openbet.com <http://www.openbet.com/>
>
>     	
>     	OpenBet Ltd
>
>     	Chiswick Park Building 9
>
>     	566 Chiswick High Rd
>
>     	London
>
>     	W4 5XT
>
>     	UK
>
>     	
>     <https://www.openbet.com/email_promo>
>
>     This message is confidential and intended only for the addressee.
>     If you have received this message in error, please immediately
>     notify the postmaster@openbet.com <ma...@openbet.com>
>     and delete it from your system as well as any copies. The content
>     of e-mails as well as traffic data may be monitored by OpenBet for
>     employment and security purposes. To protect the environment
>     please do not print this e-mail unless necessary. OpenBet Ltd.
>     Registered Office: Chiswick Park Building 9, 566 Chiswick High
>     Road, London, W4 5XT, United Kingdom. A company registered in
>     England and Wales. Registered no. 3134634. VAT no. GB927523612
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Damian Guy <da...@gmail.com>.
Hi Michal,

Thanks for the KIP. I'd like to propose a bit more of a radical change to
the API.
1. deprecate the punctuate method on Processor
2. create a new Functional Interface just for Punctuation, something like:
interface Punctuator {
    void punctuate(long timestamp)
}
3. add a new schedule function to ProcessorContext: schedule(long interval,
PunctuationType type, Punctuator callback)
4. deprecate the existing schedule function

Thoughts?

Thanks,
Damian

On Sun, 16 Apr 2017 at 21:55 Michal Borowiecki <
michal.borowiecki@openbet.com> wrote:

> Hi Thomas,
>
> I would say our use cases fall in the same category as yours.
>
> 1) One is expiry of old records, it's virtually identical to yours.
>
> 2) Second one is somewhat more convoluted but boils down to the same type
> of design:
>
> Incoming messages carry a number of fields, including a timestamp.
>
> Outgoing messages contain derived fields, one of them (X) is depended on
> by the timestamp input field (Y) and some other input field (Z).
>
> Since the output field X is derived in some non-trivial way, we don't want
> to force the logic onto downstream apps. Instead we want to calculate it in
> the Kafka Streams app, which means we re-calculate X as soon as the
> timestamp in Y is reached (wall clock time) and send a message if it
> changed (I say "if" because the derived field (X) is also conditional on
> another input field Z).
>
> So we have kv stores with the records and an additional kv store with
> timestamp->id mapping which act like an index where we periodically do a
> ranged query.
>
> Initially we naively tried doing it in punctuate which of course didn't
> work when there were no regular msgs on the input topic.
> Since this was before 0.10.1 and state stores weren't query-able from
> outside we created a "ticker" that produced msgs once per second onto
> another topic and fed it into the same topology to trigger punctuate.
> This didn't work either, which was much more surprising to us at the time,
> because it was not obvious at all that punctuate is only triggered if
> *all* input partitions receive messages regularly.
> In the end we had to break this into 2 separate Kafka Streams. Main
> transformer doesn't use punctuate but sends values of timestamp field Y and
> the id to a "scheduler" topic where also the periodic ticks are sent. This
> is consumed by the second topology and is its only input topic. There's a
> transformer on that topic which populates and updates the time-based
> indexes and polls them from punctuate. If the time in the timestamp
> elapsed, the record id is sent to the main transformer, which
> updates/deletes the record from the main kv store and forwards the
> transformed record to the output topic.
>
> To me this setup feels horrendously complicated for what it does.
>
> We could incrementally improve on this since 0.10.1 to poll the
> timestamp->id "index" stores from some code outside the KafkaStreams
> topology so that at least we wouldn't need the extra topic for "ticks".
> However, the ticks don't feel so hacky when you realise they give you some
> hypothetical benefits in predictability. You can reprocess the messages in
> a reproducible manner, since the topologies use event-time, just that the
> event time is simply the wall-clock time fed into a topic by the ticks. (NB
> in our use case we haven't yet found a need for this kind of reprocessing).
> To make that work though, we would have to have the stream time advance
> based on the presence of msgs on the "tick" topic, regardless of the
> presence of messages on the other input topic.
>
> Same as in the expiry use case, both the wall-clock triggered punctuate
> and the hybrid would work to simplify this a lot.
>
> 3) Finally, I have a 3rd use case in the making but I'm still looking if
> we can achieve it using session windows instead. I'll keep you posted if we
> have to go with punctuate there too.
>
> Thanks,
> Michal
>
>
> On 11/04/17 20:52, Thomas Becker wrote:
>
> Here's an example that we currently have.  We have a streams processor
> that does a transform from one topic into another. One of the fields in
> the source topic record is an expiration time, and one of the functions
> of the processor is to ensure that expired records get deleted promptly
> after that time passes (typically days or weeks after the message was
> originally produced). To do that, the processor keeps a state store of
> keys and expiration times, iterates that store in punctuate(), and
> emits delete (null) records for expired items. This needs to happen at
> some minimum interval regardless of the incoming message rate of the
> source topic.
>
> In this scenario, the expiration of records is the primary function of
> punctuate, and therefore the key requirement is that the wall-clock
> measured time between punctuate calls have some upper-bound. So a pure
> wall-clock based schedule would be fine for our needs. But the proposed
> "hybrid" system would also be acceptable if that satisfies a broader
> range of use-cases.
>
> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>
> I apologize for the longer email below.  To my defense, it started
> out much
> shorter. :-)  Also, to be super-clear, I am intentionally playing
> devil's
> advocate for a number of arguments brought forth in order to help
> improve
> this KIP -- I am not implying I necessarily disagree with the
> arguments.
>
> That aside, here are some further thoughts.
>
> First, there are (at least?) two categories for actions/behavior you
> invoke
> via punctuate():
>
> 1. For internal housekeeping of your Processor or Transformer (e.g.,
> to
> periodically commit to a custom store, to do metrics/logging).  Here,
> the
> impact of punctuate is typically not observable by other processing
> nodes
> in the topology.
> 2. For controlling the emit frequency of downstream records.  Here,
> the
> punctuate is all about being observable by downstream processing
> nodes.
>
> A few releases back, we introduced record caches (DSL) and state
> store
> caches (Processor API) in KIP-63.  Here, we addressed a concern
> relating to
> (2) where some users needed to control -- here: limit -- the
> downstream
> output rate of Kafka Streams because the downstream systems/apps
> would not
> be able to keep up with the upstream output rate (Kafka scalability >
> their
> scalability).  The argument for KIP-63, which notably did not
> introduce a
> "trigger" API, was that such an interaction with downstream systems
> is an
> operational concern;  it should not impact the processing *logic* of
> your
> application, and thus we didn't want to complicate the Kafka Streams
> API,
> especially not the declarative DSL, with such operational concerns.
>
> This KIP's discussion on `punctuate()` takes us back in time (<--
> sorry, I
> couldn't resist to not make this pun :-P).  As a meta-comment, I am
> observing that our conversation is moving more and more into the
> direction
> of explicit "triggers" because, so far, I have seen only motivations
> for
> use cases in category (2), but none yet for (1)?  For example, some
> comments voiced here are about sth like "IF stream-time didn't
> trigger
> punctuate, THEN trigger punctuate based on processing-time".  Do we
> want
> this, and if so, for which use cases and benefits?  Also, on a
> related
> note, whatever we are discussing here will impact state store caches
> (Processor API) and perhaps also impact record caches (DSL), thus we
> should
> clarify any such impact here.
>
> Switching topics slightly.
>
> Jay wrote:
>
>
> One thing I've always found super important for this kind of design
> work
> is to do a really good job of cataloging the landscape of use cases
> and
> how prevalent each one is.
>
> +1 to this, as others have already said.
>
> Here, let me highlight -- just in case -- that when we talked about
> windowing use cases in the recent emails, the Processor API (where
> `punctuate` resides) does not have any notion of windowing at
> all.  If you
> want to do windowing *in the Processor API*, you must do so manually
> in
> combination with window stores.  For this reason I'd suggest to
> discuss use
> cases not just in general, but also in view of how you'd do so in the
> Processor API vs. in the DSL.  Right now, changing/improving
> `punctuate`
> does not impact the DSL at all, unless we add new functionality to
> it.
>
> Jay wrote in his strawman example:
>
>
> You aggregate click and impression data for a reddit like site.
> Every ten
> minutes you want to output a ranked list of the top 10 articles
> ranked by
> clicks/impressions for each geographical area. I want to be able
> run this
> in steady state as well as rerun to regenerate results (or catch up
> if it
> crashes).
>
> This is a good example for more than the obvious reason:  In KIP-63,
> we
> argued that the reason for saying "every ten minutes" above is not
> necessarily about because you want to output data *exactly* after ten
> minutes, but that you want to perform an aggregation based on 10-
> minute
> windows of input data; i.e., the point is about specifying the input
> for
> your aggregation, not or less about when the results of the
> aggregation
> should be send downstream.  To take an extreme example, you could
> disable
> record caches and let your app output a downstream update for every
> incoming input record.  If the last input record was from at minute 7
> of 10
> (for a 10-min window), then what your app would output at minute 10
> would
> be identical to what it had already emitted at minute 7 earlier
> anyways.
> This is particularly true when we take late-arriving data into
> account:  if
> a late record arrived at minute 13, your app would (by default) send
> a new
> update downstream, even though the "original" 10 minutes have already
> passed.
>
> Jay wrote...:
>
>
> There are a couple of tricky things that seem to make this hard
> with
>
> either
>
>
> of the options proposed:
> 1. If I emit this data using event time I have the problem
> described where
> a geographical region with no new clicks or impressions will fail
> to
>
> output
>
>
> results.
>
> ...and Arun Mathew wrote:
>
>
>
> We window by the event time, but trigger punctuate in <punctuate
> interval>
> duration of system time, in the absence of an event crossing the
> punctuate
> event time.
>
> So, given what I wrote above about the status quo and what you can
> already
> do with it, is the concern that the state store cache doesn't give
> you
> *direct* control over "forcing an output after no later than X
> seconds [of
> processing-time]" but only indirect control through a cache
> size?  (Note
> that I am not dismissing the claims why this might be helpful.)
>
> Arun Mathew wrote:
>
>
> We are using Kafka Stream for our Audit Trail, where we need to
> output the
> event counts on each topic on each cluster aggregated over a 1
> minute
> window. We have to use event time to be able to cross check the
> counts.
>
> But
>
>
> we need to trigger punctuate [aggregate event pushes] by system
> time in
>
> the
>
>
> absence of events. Otherwise the event counts for unexpired windows
> would
> be 0 which is bad.
>
> Isn't the latter -- "count would be 0" -- the problem between the
> absence
> of output vs. an output of 0, similar to the use of `Option[T]` in
> Scala
> and the difference between `None` and `Some(0)`?  That is, isn't the
> root
> cause that the downstream system interprets the absence of output in
> a
> particular way ("No output after 1 minute = I consider the output to
> be
> 0.")?  Arguably, you could also adapt the downstream system (if
> possible)
> to correctly handle the difference between absence of output vs.
> output of
> 0.  I am not implying that we shouldn't care about such a use case,
> but
> want to understand the motivation better. :-)
>
> Also, to add some perspective, in some related discussions we talked
> about
> how a Kafka Streams application should not worry or not be coupled
> unnecessarily with such interpretation specifics in a downstream
> system's
> behavior.  After all, tomorrow your app's output might be consumed by
> more
> than just this one downstream system.  Arguably, Kafka Connect rather
> than
> Kafka Streams might be the best tool to link the universes of Kafka
> and
> downstream systems, including helping to reconcile the differences in
> how
> these systems interpret changes, updates, late-arriving data,
> etc.  Kafka
> Connect would allow you to decouple the Kafka Streams app's logical
> processing from the specifics of downstream systems, thanks to
> specific
> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
> this
> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
> but
> it's currently awkward to use Connect for this", this might be a
> problem we
> can solve, too.)
>
> Switching topics slightly again.
>
> Thomas wrote:
>
>
> I'm not entirely convinced that a separate callback (option C)
> is that messy (it could just be a default method with an empty
> implementation), but if we wanted a single API to handle both
> cases,
> how about something like the following?
>
> enum Time {
>    STREAM,
>    CLOCK
> }
>
> Yeah, I am on the fence here, too.  If we use the 1-method approach,
> then
> whatever the user is doing inside this method is a black box to Kafka
> Streams (similar to how we have no idea what the user does inside a
> `foreach` -- if the function passed to `foreach` writes to external
> systems, then Kafka Streams is totally unaware of the fact).  We
> won't
> know, for example, if the stream-time action has a smaller "trigger"
> frequency than the processing-time action.  Or, we won't know whether
> the
> user custom-codes a "not later than" trigger logic ("Do X every 1-
> minute of
> stream-time or 1-minute of processing-time, whichever comes
> first").  That
> said, I am not certain yet whether we would need such knowledge
> because,
> when using the Processor API, most of the work and decisions must be
> done
> by the user anyways.  It would matter though if the concept of
> "triggers"
> were to bubble up into the DSL because in the DSL the management of
> windowing, window stores, etc. must be done automatically by Kafka
> Streams.
>
> [In any case, btw, we have the corner case where the user configured
> the
> stream-time to be processing-time (e.g. via wall-clock timestamp
> extractor), at which point both punctuate variants are based on the
> same
> time semantics / timeline.]
>
> Again, I apologize for the wall of text.  Congratulations if you made
> it
> this far. :-)
>
> More than happy to hear your thoughts!
> Michael
>
> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
> wrote:
>
>
>
> Thanks Matthias.
> Sure, will correct it right away.
>
> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
> wrote:
>
> Thanks for preparing this page!
>
> About terminology:
>
> You introduce the term "event time" -- but we should call this
> "stream
> time" -- "stream time" is whatever TimestampExtractor returns and
> this
> could be event time, ingestion time, or processing/wall-clock time.
>
> Does this make sense to you?
>
>
>
> -Matthias
>
>
> On 4/10/17 4:58 AM, Arun Mathew wrote:
>
>
> Thanks Ewen.
>
> @Michal, @all, I have created a child page to start the Use Cases
>
> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
> Punctuate+Use+Cases]. Please go through it and give your comments.
>
>
>
> @Tianji, Sorry for the delay. I am trying to make the patch
> public.
>
> --
> Arun Mathew
>
> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> <ew...@confluent.io>
> wrote:
>
>     Arun,
>
>     I've given you permission to edit the wiki. Let me know if
> you run
>
> into any
>
>
>     issues.
>
>     -Ewen
>
>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co
> rp.jp> <am...@yahoo-corp.jp>
>
> wrote:
>
>
>
>     > Thanks Michal. I don’t have the access yet [arunmathew88].
> Should I
>
> be
>
>
>     > sending a separate mail for this?
>     >
>     > I thought one of the person following this thread would be
> able to
>
> give me
>
>
>     > access.
>     >
>     >
>     >
>     > *From: *Michal Borowiecki <mi...@openbet.com> <mi...@openbet.com>
>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>     > *Date: *Friday, April 7, 2017 at 17:16
>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org> <de...@kafka.apache.org> <de...@kafka.apache.org>
>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
> semantics
>     >
>     >
>     >
>     > Hi Arun,
>     >
>     > I was thinking along the same lines as you, listing the use
> cases
>
> on the
>
>
>     > wiki, but didn't find time to get around doing that yet.
>     > Don't mind if you do it if you have access now.
>     > I was thinking it would be nice if, once we have the use
> cases
>
> listed,
>
>
>     > people could use likes to up-vote the use cases similar to
> what
>
> they're
>
>
>     > working on.
>     >
>     > I should have a bit more time to action this in the next
> few days,
>
> but
>
>
>     > happy for you to do it if you can beat me to it ;-)
>     >
>     > Cheers,
>     > Michal
>     >
>     > On 07/04/17 04:39, Arun Mathew wrote:
>     >
>     > Sure, Thanks Matthias. My id is [arunmathew88].
>     >
>     >
>     >
>     > Of course. I was thinking of a subpage where people can
>
> collaborate.
>
>
>     >
>     >
>     >
>     > Will do as per Michael’s suggestion.
>     >
>     >
>     >
>     > Regards,
>     >
>     > Arun Mathew
>     >
>     >
>     >
>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io>
> <
>
> matthias@confluent.io> wrote:
>
>
>     >
>     >
>     >
>     >     Please share your Wiki-ID and a committer can give you
> write
>
> access.
>
>
>     >
>     >
>     >
>     >     Btw: as you did not initiate the KIP, you should not
> change the
>
> KIP
>
>
>     >
>     >     without the permission of the original author -- in
> this case
>
> Michael.
>
>
>     >
>     >
>     >
>     >     So you might also just share your thought over the
> mailing list
>
> and
>
>
>     >
>     >     Michael can update the KIP page. Or, as an alternative,
> just
>
> create a
>
>
>     >
>     >     subpage for the KIP page.
>     >
>     >
>     >
>     >     @Michael: WDYT?
>     >
>     >
>     >
>     >
>     >
>     >     -Matthias
>     >
>     >
>     >
>     >
>     >
>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>     >
>     >     > Hi Jay,
>     >
>     >     >           Thanks for the advise, I would like to list
> down
>
> the use cases as
>
>
>     >
>     >     > per your suggestion. But it seems I don't have write
>
> permission to the
>
>
>     >
>     >     > Apache Kafka Confluent Space. Whom shall I request
> for it?
>     >
>     >     >
>     >
>     >     > Regarding your last question. We are using a patch in
> our
>
> production system
>
>
>     >
>     >     > which does exactly this.
>     >
>     >     > We window by the event time, but trigger punctuate in
>
> <punctuate interval>
>
>
>     >
>     >     > duration of system time, in the absence of an event
> crossing
>
> the punctuate
>
>
>     >
>     >     > event time.
>     >
>     >     >
>     >
>     >     > We are using Kafka Stream for our Audit Trail, where
> we need
>
> to output the
>
>
>     >
>     >     > event counts on each topic on each cluster aggregated
> over a
>
> 1 minute
>
>
>     >
>     >     > window. We have to use event time to be able to cross
> check
>
> the counts. But
>
>
>     >
>     >     > we need to trigger punctuate [aggregate event pushes]
> by
>
> system time in the
>
>
>     >
>     >     > absence of events. Otherwise the event counts for
> unexpired
>
> windows would
>
>
>     >
>     >     > be 0 which is bad.
>     >
>     >     >
>     >
>     >     > "Maybe a hybrid solution works: I window by event
> time but
>
> trigger results
>
>
>     >
>     >     > by system time for windows that have updated? Not
> really sure
>
> the details
>
>
>     >
>     >     > of making that work. Does that work? Are there
> concrete
>
> examples where you
>
>
>     >
>     >     > actually want the current behavior?"
>     >
>     >     >
>     >
>     >     > --
>     >
>     >     > With Regards,
>     >
>     >     >
>     >
>     >     > Arun Mathew
>     >
>     >     > Yahoo! JAPAN Corporation
>     >
>     >     >
>     >
>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>
> skyahead@gmail.com><sk...@gmail.com> <sk...@gmail.com> wrote:
>
>
>     >
>     >     >
>     >
>     >     >> Hi Jay,
>     >
>     >     >>
>     >
>     >     >> The hybrid solution is exactly what I expect and
> need for
>
> our use cases
>
>
>     >
>     >     >> when dealing with telecom data.
>     >
>     >     >>
>     >
>     >     >> Thanks
>     >
>     >     >> Tianji
>     >
>     >     >>
>     >
>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>
> jay@confluent.io><ja...@confluent.io> <ja...@confluent.io> wrote:
>
>
>     >
>     >     >>
>     >
>     >     >>> Hey guys,
>     >
>     >     >>>
>     >
>     >     >>> One thing I've always found super important for
> this kind
>
> of design work
>
>
>     >
>     >     >> is
>     >
>     >     >>> to do a really good job of cataloging the landscape
> of use
>
> cases and how
>
>
>     >
>     >     >>> prevalent each one is. By that I mean not just
> listing lots
>
> of uses, but
>
>
>     >
>     >     >>> also grouping them into categories that
> functionally need
>
> the same thing.
>
>
>     >
>     >     >>> In the absence of this it is very hard to reason
> about
>
> design proposals.
>
>
>     >
>     >     >>> From the proposals so far I think we have a lot of
>
> discussion around
>
>
>     >
>     >     >>> possible apis, but less around what the user needs
> for
>
> different use
>
>
>     >
>     >     >> cases
>     >
>     >     >>> and how they would implement that using the api.
>     >
>     >     >>>
>     >
>     >     >>> Here is an example:
>     >
>     >     >>> You aggregate click and impression data for a
> reddit like
>
> site. Every ten
>
>
>     >
>     >     >>> minutes you want to output a ranked list of the top
> 10
>
> articles ranked by
>
>
>     >
>     >     >>> clicks/impressions for each geographical area. I
> want to be
>
> able run this
>
>
>     >
>     >     >>> in steady state as well as rerun to regenerate
> results (or
>
> catch up if it
>
>
>     >
>     >     >>> crashes).
>     >
>     >     >>>
>     >
>     >     >>> There are a couple of tricky things that seem to
> make this
>
> hard with
>
>
>     >
>     >     >> either
>     >
>     >     >>> of the options proposed:
>     >
>     >     >>> 1. If I emit this data using event time I have the
> problem
>
> described
>
>
>     >
>     >     >> where
>     >
>     >     >>> a geographical region with no new clicks or
> impressions
>
> will fail to
>
>
>     >
>     >     >> output
>     >
>     >     >>> results.
>     >
>     >     >>> 2. If I emit this data using system time I have the
> problem
>
> that when
>
>
>     >
>     >     >>> reprocessing data my window may not be ten minutes
> but 10
>
> hours if my
>
>
>     >
>     >     >>> processing is very fast so it dramatically changes
> the
>
> output.
>
>
>     >
>     >     >>>
>     >
>     >     >>> Maybe a hybrid solution works: I window by event
> time but
>
> trigger results
>
>
>     >
>     >     >>> by system time for windows that have updated? Not
> really
>
> sure the details
>
>
>     >
>     >     >>> of making that work. Does that work? Are there
> concrete
>
> examples where
>
>
>     >
>     >     >> you
>     >
>     >     >>> actually want the current behavior?
>     >
>     >     >>>
>     >
>     >     >>> -Jay
>     >
>     >     >>>
>     >
>     >     >>>
>     >
>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>
> arunmathew88@gmail.com> <ar...@gmail.com> <ar...@gmail.com>
>
>
>     >
>     >     >>> wrote:
>     >
>     >     >>>
>     >
>     >     >>>> Hi All,
>     >
>     >     >>>>
>     >
>     >     >>>> Thanks for the KIP. We were also in need of a
> mechanism to
>
> trigger
>
>
>     >
>     >     >>>> punctuate in the absence of events.
>     >
>     >     >>>>
>     >
>     >     >>>> As I described in [
>     >
>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>     >
>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>     >
>     >     >>>> plugin.system.issuetabpanels:comment-
> tabpanel#comment-
>
> 15926036
>
>
>     >
>     >     >>>> ],
>     >
>     >     >>>>
>     >
>     >     >>>>    - Our approached involved using the event time
> by
>
> default.
>
>
>     >
>     >     >>>>    - The method to check if there is any punctuate
> ready
>
> in the
>
>
>     >
>     >     >>>>    PunctuationQueue is triggered via the any event
>
> received by the
>
>
>     >
>     >     >> stream
>     >
>     >     >>>>    tread, or at the polling intervals in the
> absence of
>
> any events.
>
>
>     >
>     >     >>>>    - When we create Punctuate objects (which
> contains the
>
> next event
>
>
>     >
>     >     >> time
>     >
>     >     >>>>    for punctuation and interval), we also record
> the
>
> creation time
>
>
>     >
>     >     >>> (system
>     >
>     >     >>>>    time).
>     >
>     >     >>>>    - While checking for maturity of Punctuate
> Schedule by
>     >
>     >     >> mayBePunctuate
>     >
>     >     >>>>    method, we also check if the system clock has
> elapsed
>
> the punctuate
>
>
>     >
>     >     >>>>    interval since the schedule creation time.
>     >
>     >     >>>>    - In the absence of any event, or in the
> absence of any
>
> event for
>
>
>     >
>     >     >> one
>     >
>     >     >>>>    topic in the partition group assigned to the
> stream
>
> task, the system
>
>
>     >
>     >     >>>> time
>     >
>     >     >>>>    will elapse the interval and we trigger a
> punctuate
>
> using the
>
>
>     >
>     >     >> expected
>     >
>     >     >>>>    punctuation event time.
>     >
>     >     >>>>    - we then create the next punctuation schedule
> as
>
> punctuation event
>
>
>     >
>     >     >>> time
>     >
>     >     >>>>    + punctuation interval, [again recording the
> system
>
> time of creation
>
>
>     >
>     >     >>> of
>     >
>     >     >>>> the
>     >
>     >     >>>>    schedule].
>     >
>     >     >>>>
>     >
>     >     >>>> We call this a Hybrid Punctuate. Of course, this
> approach
>
> has pros and
>
>
>     >
>     >     >>>> cons.
>     >
>     >     >>>> Pros
>     >
>     >     >>>>
>     >
>     >     >>>>    - Punctuates will happen in <punctuate
> interval> time
>
> duration at
>
>
>     >
>     >     >> max
>     >
>     >     >>> in
>     >
>     >     >>>>    terms of system time.
>     >
>     >     >>>>    - The semantics as a whole continues to revolve
> around
>
> event time.
>
>
>     >
>     >     >>>>    - We can use the old data [old timestamps] to
> rerun any
>
> experiments
>
>
>     >
>     >     >> or
>     >
>     >     >>>>    tests.
>     >
>     >     >>>>
>     >
>     >     >>>> Cons
>     >
>     >     >>>>
>     >
>     >     >>>>    - In case the  <punctuate interval> is not a
> time
>
> duration [say
>
>
>     >
>     >     >>> logical
>     >
>     >     >>>>    time/event count], then the approach might not
> be
>
> meaningful.
>
>
>     >
>     >     >>>>    - In case there is a case where we have to wait
> for an
>
> actual event
>
>
>     >
>     >     >>> from
>     >
>     >     >>>>    a low event rate partition in the partition
> group, this
>
> approach
>
>
>     >
>     >     >> will
>     >
>     >     >>>> jump
>     >
>     >     >>>>    the gun.
>     >
>     >     >>>>    - in case the event processing cannot catch up
> with the
>
> event rate
>
>
>     >
>     >     >> and
>     >
>     >     >>>>    the expected timestamp events gets queued for
> long
>
> time, this
>
>
>     >
>     >     >> approach
>     >
>     >     >>>>    might jump the gun.
>     >
>     >     >>>>
>     >
>     >     >>>> I believe the above approach and discussion goes
> close to
>
> the approach
>
>
>     >
>     >     >> A.
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I like the idea of having an even count based
> punctuate.
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I agree with the discussion around approach C,
> that we
>
> should provide
>
>
>     >
>     >     >> the
>     >
>     >     >>>> user with the option to choose system time or
> event time
>
> based
>
>
>     >
>     >     >>> punctuates.
>     >
>     >     >>>> But I believe that the user predominantly wants to
> use
>
> event time while
>
>
>     >
>     >     >>> not
>     >
>     >     >>>> missing out on regular punctuates due to event
> delays or
>
> event
>
>
>     >
>     >     >> absences.
>     >
>     >     >>>> Hence a complex punctuate option as Matthias
> mentioned
>
> (quoted below)
>
>
>     >
>     >     >>> would
>     >
>     >     >>>> be most apt.
>     >
>     >     >>>>
>     >
>     >     >>>> "- We might want to add "complex" schedules later
> on
>
> (like, punctuate
>
>
>     >
>     >     >> on
>     >
>     >     >>>> every 10 seconds event-time or 60 seconds system-
> time
>
> whatever comes
>
>
>     >
>     >     >>>> first)."
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I think I read somewhere that Kafka Streams
> started with
>
> System Time as
>
>
>     >
>     >     >>> the
>     >
>     >     >>>> punctuation standard, but was later changed to
> Event Time.
>
> I guess
>
>
>     >
>     >     >> there
>     >
>     >     >>>> would be some good reason behind it. As Kafka
> Streams want
>
> to evolve
>
>
>     >
>     >     >> more
>     >
>     >     >>>> on the Stream Processing front, I believe the
> emphasis on
>
> event time
>
>
>     >
>     >     >>> would
>     >
>     >     >>>> remain quite strong.
>     >
>     >     >>>>
>     >
>     >     >>>>
>     >
>     >     >>>> With Regards,
>     >
>     >     >>>>
>     >
>     >     >>>> Arun Mathew
>     >
>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>     >
>     >     >>>>
>     >
>     >     >>>>
>     >
>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>
> tobecker@tivo.com> <to...@tivo.com> <to...@tivo.com>
>
>
>     >
>     >     >> wrote:
>     >
>     >     >>>>
>     >
>     >     >>>>> Yeah I like PuncutationType much better; I just
> threw
>
> Time out there
>
>
>     >
>     >     >>>>> more as a strawman than an actual suggestion ;) I
> still
>
> think it's
>
>
>     >
>     >     >>>>> worth considering what this buys us over an
> additional
>
> callback. I
>
>
>     >
>     >     >>>>> foresee a number of punctuate implementations
> following
>
> this pattern:
>
>
>     >
>     >     >>>>>
>     >
>     >     >>>>> public void punctuate(PunctuationType type) {
>     >
>     >     >>>>>     switch (type) {
>     >
>     >     >>>>>         case EVENT_TIME:
>     >
>     >     >>>>>             methodA();
>     >
>     >     >>>>>             break;
>     >
>     >     >>>>>         case SYSTEM_TIME:
>     >
>     >     >>>>>             methodB();
>     >
>     >     >>>>>             break;
>     >
>     >     >>>>>     }
>     >
>     >     >>>>> }
>     >
>     >     >>>>>
>     >
>     >     >>>>> I guess one advantage of this approach is we
> could add
>
> additional
>
>
>     >
>     >     >>>>> punctuation types later in a backwards compatible
> way
>
> (like event
>
>
>     >
>     >     >> count
>     >
>     >     >>>>> as you mentioned).
>     >
>     >     >>>>>
>     >
>     >     >>>>> -Tommy
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
> Sax wrote:
>     >
>     >     >>>>>> That sounds promising.
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> I am just wondering if `Time` is the best name.
> Maybe we
>
> want to
>
>
>     >
>     >     >> add
>     >
>     >     >>>>>> other non-time based punctuations at some point
> later. I
>
> would
>
>
>     >
>     >     >>>>>> suggest
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> enum PunctuationType {
>     >
>     >     >>>>>>   EVENT_TIME,
>     >
>     >     >>>>>>   SYSTEM_TIME,
>     >
>     >     >>>>>> }
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> or similar. Just to keep the door open -- it's
> easier to
>
> add new
>
>
>     >
>     >     >>>>>> stuff
>     >
>     >     >>>>>> if the name is more generic.
>     >
>     >     >>>>>>
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> -Matthias
>     >
>     >     >>>>>>
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> I agree that the framework providing and
> managing the
>
> notion of
>
>
>     >
>     >     >>>>>>> stream
>     >
>     >     >>>>>>> time is valuable and not something we would
> want to
>
> delegate to
>
>
>     >
>     >     >> the
>     >
>     >     >>>>>>> tasks. I'm not entirely convinced that a
> separate
>
> callback
>
>
>     >
>     >     >> (option
>     >
>     >     >>>>>>> C)
>     >
>     >     >>>>>>> is that messy (it could just be a default
> method with
>
> an empty
>
>
>     >
>     >     >>>>>>> implementation), but if we wanted a single API
> to
>
> handle both
>
>
>     >
>     >     >>>>>>> cases,
>     >
>     >     >>>>>>> how about something like the following?
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> enum Time {
>     >
>     >     >>>>>>>    STREAM,
>     >
>     >     >>>>>>>    CLOCK
>     >
>     >     >>>>>>> }
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Then on ProcessorContext:
>     >
>     >     >>>>>>> context.schedule(Time time, long interval)  //
> We could
>
> allow
>
>
>     >
>     >     >> this
>     >
>     >     >>>>>>> to
>     >
>     >     >>>>>>> be called once for each value of time to mix
>
> approaches.
>
>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Then the Processor API becomes:
>     >
>     >     >>>>>>> punctuate(Time time) // time here denotes which
>
> schedule resulted
>
>
>     >
>     >     >>>>>>> in
>     >
>     >     >>>>>>> this call.
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Thoughts?
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
> Sax
>
> wrote:
>
>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> I was thinking about the four options you
> proposed in
>
> more
>
>
>     >
>     >     >>>>>>>> details
>     >
>     >     >>>>>>>> and
>     >
>     >     >>>>>>>> this are my thoughts:
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (A) You argue, that users can still
> "punctuate" on
>
> event-time
>
>
>     >
>     >     >> via
>     >
>     >     >>>>>>>> process(), but I am not sure if this is
> possible.
>
> Note, that
>
>
>     >
>     >     >>>>>>>> users
>     >
>     >     >>>>>>>> only
>     >
>     >     >>>>>>>> get record timestamps via context.timestamp().
> Thus,
>
> users
>
>
>     >
>     >     >> would
>     >
>     >     >>>>>>>> need
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> track the time progress per partition (based
> on the
>
> partitions
>
>
>     >
>     >     >>>>>>>> they
>     >
>     >     >>>>>>>> obverse via context.partition(). (This alone
> puts a
>
> huge burden
>
>
>     >
>     >     >>>>>>>> on
>     >
>     >     >>>>>>>> the
>     >
>     >     >>>>>>>> user by itself.) However, users are not
> notified at
>
> startup
>
>
>     >
>     >     >> what
>     >
>     >     >>>>>>>> partitions are assigned, and user are not
> notified
>
> when
>
>
>     >
>     >     >>>>>>>> partitions
>     >
>     >     >>>>>>>> get
>     >
>     >     >>>>>>>> revoked. Because this information is not
> available,
>
> it's not
>
>
>     >
>     >     >>>>>>>> possible
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> "manually advance" stream-time, and thus
> event-time
>
> punctuation
>
>
>     >
>     >     >>>>>>>> within
>     >
>     >     >>>>>>>> process() seems not to be possible -- or do
> you see a
>
> way to
>
>
>     >
>     >     >> get
>     >
>     >     >>>>>>>> it
>     >
>     >     >>>>>>>> done? And even if, it might still be too
> clumsy to
>
> use.
>
>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (B) This does not allow to mix both
> approaches, thus
>
> limiting
>
>
>     >
>     >     >>>>>>>> what
>     >
>     >     >>>>>>>> users
>     >
>     >     >>>>>>>> can do.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (C) This should give all flexibility we need.
> However,
>
> just
>
>
>     >
>     >     >>>>>>>> adding
>     >
>     >     >>>>>>>> one
>     >
>     >     >>>>>>>> more method seems to be a solution that is too
> simple
>
> (cf my
>
>
>     >
>     >     >>>>>>>> comments
>     >
>     >     >>>>>>>> below).
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (D) This might be hard to use. Also, I am not
> sure how
>
> a user
>
>
>     >
>     >     >>>>>>>> could
>     >
>     >     >>>>>>>> enable system-time and event-time punctuation
> in
>
> parallel.
>
>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Overall options (C) seems to be the most
> promising
>
> approach to
>
>
>     >
>     >     >>>>>>>> me.
>     >
>     >     >>>>>>>> Because I also favor a clean API, we might
> keep
>
> current
>
>
>     >
>     >     >>>>>>>> punctuate()
>     >
>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it
> at some
>
> later
>
>
>     >
>     >     >>>>>>>> point
>     >
>     >     >>>>>>>> when
>     >
>     >     >>>>>>>> people use the "new punctuate API".
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Couple of follow up questions:
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - I am wondering, if we should have two
> callback
>
> methods or
>
>
>     >
>     >     >> just
>     >
>     >     >>>>>>>> one
>     >
>     >     >>>>>>>> (ie, a unified for system and event time
> punctuation
>
> or one for
>
>
>     >
>     >     >>>>>>>> each?).
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - If we have one, how can the user figure out,
> which
>
> condition
>
>
>     >
>     >     >>>>>>>> did
>     >
>     >     >>>>>>>> trigger?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - How would the API look like, for registering
>
> different
>
>
>     >
>     >     >>>>>>>> punctuate
>     >
>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - We might want to add "complex" schedules
> later on
>
> (like,
>
>
>     >
>     >     >>>>>>>> punctuate
>     >
>     >     >>>>>>>> on
>     >
>     >     >>>>>>>> every 10 seconds event-time or 60 seconds
> system-time
>
> whatever
>
>
>     >
>     >     >>>>>>>> comes
>     >
>     >     >>>>>>>> first). I don't say we should add this right
> away, but
>
> we might
>
>
>     >
>     >     >>>>>>>> want
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> define the API in a way, that it allows
> extensions
>
> like this
>
>
>     >
>     >     >>>>>>>> later
>     >
>     >     >>>>>>>> on,
>     >
>     >     >>>>>>>> without redesigning the API (ie, the API
> should be
>
> designed
>
>
>     >
>     >     >>>>>>>> extensible)
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - Did you ever consider count-based
> punctuation?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> I understand, that you would like to solve a
> simple
>
> problem,
>
>
>     >
>     >     >> but
>     >
>     >     >>>>>>>> we
>     >
>     >     >>>>>>>> learned from the past, that just "adding some
> API"
>
> quickly
>
>
>     >
>     >     >> leads
>     >
>     >     >>>>>>>> to a
>     >
>     >     >>>>>>>> not very well defined API that needs time
> consuming
>
> clean up
>
>
>     >
>     >     >>>>>>>> later on
>     >
>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
> holistic
>     >
>     >     >>>>>>>> punctuation
>     >
>     >     >>>>>>>> KIP
>     >
>     >     >>>>>>>> with this from the beginning on to avoid later
> painful
>     >
>     >     >> redesign.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> -Matthias
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Thanks Thomas,
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> I'm also wary of changing the existing
> semantics of
>     >
>     >     >> punctuate,
>     >
>     >     >>>>>>>>> for
>     >
>     >     >>>>>>>>> backward compatibility reasons, although I
> like the
>     >
>     >     >> conceptual
>     >
>     >     >>>>>>>>> simplicity of that option.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Adding a new method to me feels safer but, in
> a way,
>
> uglier.
>
>
>     >
>     >     >> I
>     >
>     >     >>>>>>>>> added
>     >
>     >     >>>>>>>>> this to the KIP now as option (C).
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> The TimestampExtractor mechanism is actually
> more
>
> flexible,
>
>
>     >
>     >     >> as
>     >
>     >     >>>>>>>>> it
>     >
>     >     >>>>>>>>> allows
>     >
>     >     >>>>>>>>> you to return any value, you're not limited
> to event
>
> time or
>
>
>     >
>     >     >>>>>>>>> system
>     >
>     >     >>>>>>>>> time
>     >
>     >     >>>>>>>>> (although I don't see an actual use case
> where you
>
> might need
>
>
>     >
>     >     >>>>>>>>> anything
>     >
>     >     >>>>>>>>> else then those two). Hence I also proposed
> the
>
> option to
>
>
>     >
>     >     >> allow
>     >
>     >     >>>>>>>>> users
>     >
>     >     >>>>>>>>> to, effectively, decide what "stream time" is
> for
>
> them given
>
>
>     >
>     >     >>>>>>>>> the
>     >
>     >     >>>>>>>>> presence or absence of messages, much like
> they can
>
> decide
>
>
>     >
>     >     >> what
>     >
>     >     >>>>>>>>> msg
>     >
>     >     >>>>>>>>> time
>     >
>     >     >>>>>>>>> means for them using the TimestampExtractor.
> What do
>
> you
>
>
>     >
>     >     >> think
>     >
>     >     >>>>>>>>> about
>     >
>     >     >>>>>>>>> that? This is probably most flexible but also
> most
>     >
>     >     >> complicated.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> All comments appreciated.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Cheers,
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Michal
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>> Although I fully agree we need a way to
> trigger
>
> periodic
>
>
>     >
>     >     >>>>>>>>>> processing
>     >
>     >     >>>>>>>>>> that is independent from whether and when
> messages
>
> arrive,
>
>
>     >
>     >     >>>>>>>>>> I'm
>     >
>     >     >>>>>>>>>> not sure
>     >
>     >     >>>>>>>>>> I like the idea of changing the existing
> semantics
>
> across
>
>
>     >
>     >     >> the
>     >
>     >     >>>>>>>>>> board.
>     >
>     >     >>>>>>>>>> What if we added an additional callback to
> Processor
>
> that
>
>
>     >
>     >     >> can
>     >
>     >     >>>>>>>>>> be
>     >
>     >     >>>>>>>>>> scheduled similarly to punctuate() but was
> always
>
> called at
>
>
>     >
>     >     >>>>>>>>>> fixed, wall
>     >
>     >     >>>>>>>>>> clock based intervals? This way you wouldn't
> have to
>
> give
>
>
>     >
>     >     >> up
>     >
>     >     >>>>>>>>>> the
>     >
>     >     >>>>>>>>>> notion
>     >
>     >     >>>>>>>>>> of stream time to be able to do periodic
> processing.
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
> Borowiecki
>
> wrote:
>
>
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Hi all,
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>
> punctuate
>
>
>     >
>     >     >>>>>>>>>>> semantics
>     >
>     >     >>>>>>>>>>> <https://cwiki.apache.org/
>
> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
> confluence/display/KAFKA/KIP-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>
>
>     >
>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI
> P-> <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
>
> 138%
>
>
>     >
>     >     >>>>>>>>>>> 3A+C
>     >
>     >     >>>>>>>>>>> hange+
>     >
>     >     >>>>>>>>>>> punctuate+semantics>
>     >
>     >     >>>>>>>>>>> .
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Appreciating there can be different views
> on
>
> system-time
>
>
>     >
>     >     >> vs
>     >
>     >     >>>>>>>>>>> event-
>     >
>     >     >>>>>>>>>>> time
>     >
>     >     >>>>>>>>>>> semantics for punctuation depending on use-
> case and
>
> the
>
>
>     >
>     >     >>>>>>>>>>> importance of
>     >
>     >     >>>>>>>>>>> backwards compatibility of any such change,
> I've
>
> left it
>
>
>     >
>     >     >>>>>>>>>>> quite
>     >
>     >     >>>>>>>>>>> open
>     >
>     >     >>>>>>>>>>> and
>     >
>     >     >>>>>>>>>>> hope to fill in more info as the discussion
>
> progresses.
>
>
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Thanks,
>     >
>     >     >>>>>>>>>>> Michal
>     >
>     >     >>>>>>> --
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     Tommy Becker
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     Senior Software Engineer
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747> <(919)%20460-4747>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     tivo.com
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> ________________________________
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> This email and any attachments may contain
> confidential
>
> and
>
>
>     >
>     >     >>>>>>> privileged material for the sole use of the
> intended
>
> recipient.
>
>
>     >
>     >     >> Any
>     >
>     >     >>>>>>> review, copying, or distribution of this email
> (or any
>     >
>     >     >> attachments)
>     >
>     >     >>>>>>> by others is prohibited. If you are not the
> intended
>
> recipient,
>
>
>     >
>     >     >>>>>>> please contact the sender immediately and
> permanently
>
> delete this
>
>
>     >
>     >     >>>>>>> email and any attachments. No employee or agent
> of TiVo
>
> Inc. is
>
>
>     >
>     >     >>>>>>> authorized to conclude any binding agreement on
> behalf
>
> of TiVo
>
>
>     >
>     >     >> Inc.
>     >
>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may
> only be
>
> made by a
>
>
>     >
>     >     >>>>>>> signed written agreement.
>     >
>     >     >>>>>>>
>     >
>     >     >>>>> --
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>>     Tommy Becker
>     >
>     >     >>>>>
>     >
>     >     >>>>>     Senior Software Engineer
>     >
>     >     >>>>>
>     >
>     >     >>>>>     O +1 919.460.4747 <(919)%20460-4747> <(919)%20460-4747>
>     >
>     >     >>>>>
>     >
>     >     >>>>>     tivo.com
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>> ________________________________
>     >
>     >     >>>>>
>     >
>     >     >>>>> This email and any attachments may contain
> confidential
>
> and
>
>
>     >
>     >     >> privileged
>     >
>     >     >>>>> material for the sole use of the intended
> recipient. Any
>
> review,
>
>
>     >
>     >     >>> copying,
>     >
>     >     >>>>> or distribution of this email (or any
> attachments) by
>
> others is
>
>
>     >
>     >     >>>> prohibited.
>     >
>     >     >>>>> If you are not the intended recipient, please
> contact the
>
> sender
>
>
>     >
>     >     >>>>> immediately and permanently delete this email and
> any
>
> attachments. No
>
>
>     >
>     >     >>>>> employee or agent of TiVo Inc. is authorized to
> conclude
>
> any binding
>
>
>     >
>     >     >>>>> agreement on behalf of TiVo Inc. by email.
> Binding
>
> agreements with
>
>
>     >
>     >     >> TiVo
>     >
>     >     >>>>> Inc. may only be made by a signed written
> agreement.
>     >
>     >     >>>>>
>     >
>     >     >>>>
>     >
>     >     >>>
>     >
>     >     >>
>     >
>     >     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     > --
>     >
>     > <http://www.openbet.com/> <http://www.openbet.com/>
>
>
>     >
>     > *Michal Borowiecki*
>     >
>     > *Senior Software Engineer L4*
>     >
>     > *T: *
>     >
>     > +44 208 742 1600 <+44%2020%208742%201600> <+44%2020%208742%201600>
>     >
>     > +44 203 249 8448 <+44%2020%203249%208448> <+44%2020%203249%208448>
>     >
>     >
>     >
>     > *E: *
>     >
>     > michal.borowiecki@openbet.com
>     >
>     > *W: *
>     >
>     > www.openbet.com
>     >
>     > *OpenBet Ltd*
>     >
>     > Chiswick Park Building 9
>     >
>     > 566 Chiswick High Rd
>     >
>     > London
>     >
>     > W4 5XT
>     >
>     > UK
>     >
>     > <https://www.openbet.com/email_promo> <https://www.openbet.com/email_promo>
>     >
>     >
>     >
>     > This message is confidential and intended only for the
> addressee.
>
> If you
>
>
>     > have received this message in error, please immediately
> notify the
>     > postmaster@openbet.com and delete it from your system as
> well as
>
> any
>
>
>     > copies. The content of e-mails as well as traffic data may
> be
>
> monitored by
>
>
>     > OpenBet for employment and security purposes. To protect
> the
>
> environment
>
>
>     > please do not print this e-mail unless necessary. OpenBet
> Ltd.
>
> Registered
>
>
>     > Office: Chiswick Park Building 9, 566 Chiswick High Road,
> London,
>
> W4 5XT,
>
>
>     > United Kingdom. A company registered in England and Wales.
>
> Registered no.
>
>
>     > 3134634. VAT no. GB927523612
>     >
>     >
>     >
>
>
>
> --
>
>
>     Tommy Becker
>
>     Senior Software Engineer
>
>     O +1 919.460.4747 <(919)%20460-4747>
>
>
>
>     tivo.com
>
>
>
>
> ________________________________
>
> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
>
>
> --
> <http://www.openbet.com/> Michal Borowiecki
> Senior Software Engineer L4
> T: +44 208 742 1600 <+44%2020%208742%201600>
>
>
> +44 203 249 8448 <+44%2020%203249%208448>
>
>
>
> E: michal.borowiecki@openbet.com
> W: www.openbet.com
> OpenBet Ltd
>
> Chiswick Park Building 9
>
> 566 Chiswick High Rd
>
> London
>
> W4 5XT
>
> UK
> <https://www.openbet.com/email_promo>
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com and delete it from your system as well as any
> copies. The content of e-mails as well as traffic data may be monitored by
> OpenBet for employment and security purposes. To protect the environment
> please do not print this e-mail unless necessary. OpenBet Ltd. Registered
> Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
> United Kingdom. A company registered in England and Wales. Registered no.
> 3134634. VAT no. GB927523612
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi Thomas,

I would say our use cases fall in the same category as yours.

1) One is expiry of old records, it's virtually identical to yours.

2) Second one is somewhat more convoluted but boils down to the same 
type of design:

Incoming messages carry a number of fields, including a timestamp.

Outgoing messages contain derived fields, one of them (X) is depended on 
by the timestamp input field (Y) and some other input field (Z).

Since the output field X is derived in some non-trivial way, we don't 
want to force the logic onto downstream apps. Instead we want to 
calculate it in the Kafka Streams app, which means we re-calculate X as 
soon as the timestamp in Y is reached (wall clock time) and send a 
message if it changed (I say "if" because the derived field (X) is also 
conditional on another input field Z).


So we have kv stores with the records and an additional kv store with 
timestamp->id mapping which act like an index where we periodically do a 
ranged query.

Initially we naively tried doing it in punctuate which of course didn't 
work when there were no regular msgs on the input topic.
Since this was before 0.10.1 and state stores weren't query-able from 
outside we created a "ticker" that produced msgs once per second onto 
another topic and fed it into the same topology to trigger punctuate.
This didn't work either, which was much more surprising to us at the 
time, because it was not obvious at all that punctuate is only triggered 
if /*all*/ input partitions receive messages regularly.
In the end we had to break this into 2 separate Kafka Streams. Main 
transformer doesn't use punctuate but sends values of timestamp field Y 
and the id to a "scheduler" topic where also the periodic ticks are 
sent. This is consumed by the second topology and is its only input 
topic. There's a transformer on that topic which populates and updates 
the time-based indexes and polls them from punctuate. If the time in the 
timestamp elapsed, the record id is sent to the main transformer, which 
updates/deletes the record from the main kv store and forwards the 
transformed record to the output topic.

To me this setup feels horrendously complicated for what it does.

We could incrementally improve on this since 0.10.1 to poll the 
timestamp->id "index" stores from some code outside the KafkaStreams 
topology so that at least we wouldn't need the extra topic for "ticks".
However, the ticks don't feel so hacky when you realise they give you 
some hypothetical benefits in predictability. You can reprocess the 
messages in a reproducible manner, since the topologies use event-time, 
just that the event time is simply the wall-clock time fed into a topic 
by the ticks. (NB in our use case we haven't yet found a need for this 
kind of reprocessing).
To make that work though, we would have to have the stream time advance 
based on the presence of msgs on the "tick" topic, regardless of the 
presence of messages on the other input topic.

Same as in the expiry use case, both the wall-clock triggered punctuate 
and the hybrid would work to simplify this a lot.

3) Finally, I have a 3rd use case in the making but I'm still looking if 
we can achieve it using session windows instead. I'll keep you posted if 
we have to go with punctuate there too.

Thanks,
Michal

On 11/04/17 20:52, Thomas Becker wrote:
> Here's an example that we currently have.  We have a streams processor
> that does a transform from one topic into another. One of the fields in
> the source topic record is an expiration time, and one of the functions
> of the processor is to ensure that expired records get deleted promptly
> after that time passes (typically days or weeks after the message was
> originally produced). To do that, the processor keeps a state store of
> keys and expiration times, iterates that store in punctuate(), and
> emits delete (null) records for expired items. This needs to happen at
> some minimum interval regardless of the incoming message rate of the
> source topic.
>
> In this scenario, the expiration of records is the primary function of
> punctuate, and therefore the key requirement is that the wall-clock
> measured time between punctuate calls have some upper-bound. So a pure
> wall-clock based schedule would be fine for our needs. But the proposed
> "hybrid" system would also be acceptable if that satisfies a broader
> range of use-cases.
>
> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>> I apologize for the longer email below.  To my defense, it started
>> out much
>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>> devil's
>> advocate for a number of arguments brought forth in order to help
>> improve
>> this KIP -- I am not implying I necessarily disagree with the
>> arguments.
>>
>> That aside, here are some further thoughts.
>>
>> First, there are (at least?) two categories for actions/behavior you
>> invoke
>> via punctuate():
>>
>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>> to
>> periodically commit to a custom store, to do metrics/logging).  Here,
>> the
>> impact of punctuate is typically not observable by other processing
>> nodes
>> in the topology.
>> 2. For controlling the emit frequency of downstream records.  Here,
>> the
>> punctuate is all about being observable by downstream processing
>> nodes.
>>
>> A few releases back, we introduced record caches (DSL) and state
>> store
>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>> relating to
>> (2) where some users needed to control -- here: limit -- the
>> downstream
>> output rate of Kafka Streams because the downstream systems/apps
>> would not
>> be able to keep up with the upstream output rate (Kafka scalability >
>> their
>> scalability).  The argument for KIP-63, which notably did not
>> introduce a
>> "trigger" API, was that such an interaction with downstream systems
>> is an
>> operational concern;  it should not impact the processing *logic* of
>> your
>> application, and thus we didn't want to complicate the Kafka Streams
>> API,
>> especially not the declarative DSL, with such operational concerns.
>>
>> This KIP's discussion on `punctuate()` takes us back in time (<--
>> sorry, I
>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>> observing that our conversation is moving more and more into the
>> direction
>> of explicit "triggers" because, so far, I have seen only motivations
>> for
>> use cases in category (2), but none yet for (1)?  For example, some
>> comments voiced here are about sth like "IF stream-time didn't
>> trigger
>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>> want
>> this, and if so, for which use cases and benefits?  Also, on a
>> related
>> note, whatever we are discussing here will impact state store caches
>> (Processor API) and perhaps also impact record caches (DSL), thus we
>> should
>> clarify any such impact here.
>>
>> Switching topics slightly.
>>
>> Jay wrote:
>>> One thing I've always found super important for this kind of design
>>> work
>>> is to do a really good job of cataloging the landscape of use cases
>>> and
>>> how prevalent each one is.
>> +1 to this, as others have already said.
>>
>> Here, let me highlight -- just in case -- that when we talked about
>> windowing use cases in the recent emails, the Processor API (where
>> `punctuate` resides) does not have any notion of windowing at
>> all.  If you
>> want to do windowing *in the Processor API*, you must do so manually
>> in
>> combination with window stores.  For this reason I'd suggest to
>> discuss use
>> cases not just in general, but also in view of how you'd do so in the
>> Processor API vs. in the DSL.  Right now, changing/improving
>> `punctuate`
>> does not impact the DSL at all, unless we add new functionality to
>> it.
>>
>> Jay wrote in his strawman example:
>>> You aggregate click and impression data for a reddit like site.
>>> Every ten
>>> minutes you want to output a ranked list of the top 10 articles
>>> ranked by
>>> clicks/impressions for each geographical area. I want to be able
>>> run this
>>> in steady state as well as rerun to regenerate results (or catch up
>>> if it
>>> crashes).
>> This is a good example for more than the obvious reason:  In KIP-63,
>> we
>> argued that the reason for saying "every ten minutes" above is not
>> necessarily about because you want to output data *exactly* after ten
>> minutes, but that you want to perform an aggregation based on 10-
>> minute
>> windows of input data; i.e., the point is about specifying the input
>> for
>> your aggregation, not or less about when the results of the
>> aggregation
>> should be send downstream.  To take an extreme example, you could
>> disable
>> record caches and let your app output a downstream update for every
>> incoming input record.  If the last input record was from at minute 7
>> of 10
>> (for a 10-min window), then what your app would output at minute 10
>> would
>> be identical to what it had already emitted at minute 7 earlier
>> anyways.
>> This is particularly true when we take late-arriving data into
>> account:  if
>> a late record arrived at minute 13, your app would (by default) send
>> a new
>> update downstream, even though the "original" 10 minutes have already
>> passed.
>>
>> Jay wrote...:
>>> There are a couple of tricky things that seem to make this hard
>>> with
>> either
>>> of the options proposed:
>>> 1. If I emit this data using event time I have the problem
>>> described where
>>> a geographical region with no new clicks or impressions will fail
>>> to
>> output
>>> results.
>> ...and Arun Mathew wrote:
>>
>>> We window by the event time, but trigger punctuate in <punctuate
>>> interval>
>>> duration of system time, in the absence of an event crossing the
>>> punctuate
>>> event time.
>> So, given what I wrote above about the status quo and what you can
>> already
>> do with it, is the concern that the state store cache doesn't give
>> you
>> *direct* control over "forcing an output after no later than X
>> seconds [of
>> processing-time]" but only indirect control through a cache
>> size?  (Note
>> that I am not dismissing the claims why this might be helpful.)
>>
>> Arun Mathew wrote:
>>> We are using Kafka Stream for our Audit Trail, where we need to
>>> output the
>>> event counts on each topic on each cluster aggregated over a 1
>>> minute
>>> window. We have to use event time to be able to cross check the
>>> counts.
>> But
>>> we need to trigger punctuate [aggregate event pushes] by system
>>> time in
>> the
>>> absence of events. Otherwise the event counts for unexpired windows
>>> would
>>> be 0 which is bad.
>> Isn't the latter -- "count would be 0" -- the problem between the
>> absence
>> of output vs. an output of 0, similar to the use of `Option[T]` in
>> Scala
>> and the difference between `None` and `Some(0)`?  That is, isn't the
>> root
>> cause that the downstream system interprets the absence of output in
>> a
>> particular way ("No output after 1 minute = I consider the output to
>> be
>> 0.")?  Arguably, you could also adapt the downstream system (if
>> possible)
>> to correctly handle the difference between absence of output vs.
>> output of
>> 0.  I am not implying that we shouldn't care about such a use case,
>> but
>> want to understand the motivation better. :-)
>>
>> Also, to add some perspective, in some related discussions we talked
>> about
>> how a Kafka Streams application should not worry or not be coupled
>> unnecessarily with such interpretation specifics in a downstream
>> system's
>> behavior.  After all, tomorrow your app's output might be consumed by
>> more
>> than just this one downstream system.  Arguably, Kafka Connect rather
>> than
>> Kafka Streams might be the best tool to link the universes of Kafka
>> and
>> downstream systems, including helping to reconcile the differences in
>> how
>> these systems interpret changes, updates, late-arriving data,
>> etc.  Kafka
>> Connect would allow you to decouple the Kafka Streams app's logical
>> processing from the specifics of downstream systems, thanks to
>> specific
>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>> this
>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>> but
>> it's currently awkward to use Connect for this", this might be a
>> problem we
>> can solve, too.)
>>
>> Switching topics slightly again.
>>
>> Thomas wrote:
>>> I'm not entirely convinced that a separate callback (option C)
>>> is that messy (it could just be a default method with an empty
>>> implementation), but if we wanted a single API to handle both
>>> cases,
>>> how about something like the following?
>>>
>>> enum Time {
>>>     STREAM,
>>>     CLOCK
>>> }
>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>> then
>> whatever the user is doing inside this method is a black box to Kafka
>> Streams (similar to how we have no idea what the user does inside a
>> `foreach` -- if the function passed to `foreach` writes to external
>> systems, then Kafka Streams is totally unaware of the fact).  We
>> won't
>> know, for example, if the stream-time action has a smaller "trigger"
>> frequency than the processing-time action.  Or, we won't know whether
>> the
>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>> minute of
>> stream-time or 1-minute of processing-time, whichever comes
>> first").  That
>> said, I am not certain yet whether we would need such knowledge
>> because,
>> when using the Processor API, most of the work and decisions must be
>> done
>> by the user anyways.  It would matter though if the concept of
>> "triggers"
>> were to bubble up into the DSL because in the DSL the management of
>> windowing, window stores, etc. must be done automatically by Kafka
>> Streams.
>>
>> [In any case, btw, we have the corner case where the user configured
>> the
>> stream-time to be processing-time (e.g. via wall-clock timestamp
>> extractor), at which point both punctuate variants are based on the
>> same
>> time semantics / timeline.]
>>
>> Again, I apologize for the wall of text.  Congratulations if you made
>> it
>> this far. :-)
>>
>> More than happy to hear your thoughts!
>> Michael
>>
>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com>
>> wrote:
>>
>>> Thanks Matthias.
>>> Sure, will correct it right away.
>>>
>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io>
>>> wrote:
>>>
>>> Thanks for preparing this page!
>>>
>>> About terminology:
>>>
>>> You introduce the term "event time" -- but we should call this
>>> "stream
>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>> this
>>> could be event time, ingestion time, or processing/wall-clock time.
>>>
>>> Does this make sense to you?
>>>
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>> Thanks Ewen.
>>>>
>>>> @Michal, @all, I have created a child page to start the Use Cases
>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>
>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>> public.
>>>>
>>>> --
>>>> Arun Mathew
>>>>
>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io>
>>>> wrote:
>>>>
>>>>      Arun,
>>>>
>>>>      I've given you permission to edit the wiki. Let me know if
>>>> you run
>>> into any
>>>>      issues.
>>>>
>>>>      -Ewen
>>>>
>>>>      On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co
>>>> rp.jp>
>>> wrote:
>>>>
>>>>      > Thanks Michal. I don\u2019t have the access yet [arunmathew88].
>>>> Should I
>>> be
>>>>      > sending a separate mail for this?
>>>>      >
>>>>      > I thought one of the person following this thread would be
>>>> able to
>>> give me
>>>>      > access.
>>>>      >
>>>>      >
>>>>      >
>>>>      > *From: *Michal Borowiecki <mi...@openbet.com>
>>>>      > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>>>>      > *Date: *Friday, April 7, 2017 at 17:16
>>>>      > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>>>>      > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>> semantics
>>>>      >
>>>>      >
>>>>      >
>>>>      > Hi Arun,
>>>>      >
>>>>      > I was thinking along the same lines as you, listing the use
>>>> cases
>>> on the
>>>>      > wiki, but didn't find time to get around doing that yet.
>>>>      > Don't mind if you do it if you have access now.
>>>>      > I was thinking it would be nice if, once we have the use
>>>> cases
>>> listed,
>>>>      > people could use likes to up-vote the use cases similar to
>>>> what
>>> they're
>>>>      > working on.
>>>>      >
>>>>      > I should have a bit more time to action this in the next
>>>> few days,
>>> but
>>>>      > happy for you to do it if you can beat me to it ;-)
>>>>      >
>>>>      > Cheers,
>>>>      > Michal
>>>>      >
>>>>      > On 07/04/17 04:39, Arun Mathew wrote:
>>>>      >
>>>>      > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>      >
>>>>      >
>>>>      >
>>>>      > Of course. I was thinking of a subpage where people can
>>> collaborate.
>>>>      >
>>>>      >
>>>>      >
>>>>      > Will do as per Michael\u2019s suggestion.
>>>>      >
>>>>      >
>>>>      >
>>>>      > Regards,
>>>>      >
>>>>      > Arun Mathew
>>>>      >
>>>>      >
>>>>      >
>>>>      > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io>
>>>> <
>>> matthias@confluent.io> wrote:
>>>>      >
>>>>      >
>>>>      >
>>>>      >     Please share your Wiki-ID and a committer can give you
>>>> write
>>> access.
>>>>      >
>>>>      >
>>>>      >
>>>>      >     Btw: as you did not initiate the KIP, you should not
>>>> change the
>>> KIP
>>>>      >
>>>>      >     without the permission of the original author -- in
>>>> this case
>>> Michael.
>>>>      >
>>>>      >
>>>>      >
>>>>      >     So you might also just share your thought over the
>>>> mailing list
>>> and
>>>>      >
>>>>      >     Michael can update the KIP page. Or, as an alternative,
>>>> just
>>> create a
>>>>      >
>>>>      >     subpage for the KIP page.
>>>>      >
>>>>      >
>>>>      >
>>>>      >     @Michael: WDYT?
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >     -Matthias
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>      >
>>>>      >     > Hi Jay,
>>>>      >
>>>>      >     >           Thanks for the advise, I would like to list
>>>> down
>>> the use cases as
>>>>      >
>>>>      >     > per your suggestion. But it seems I don't have write
>>> permission to the
>>>>      >
>>>>      >     > Apache Kafka Confluent Space. Whom shall I request
>>>> for it?
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > Regarding your last question. We are using a patch in
>>>> our
>>> production system
>>>>      >
>>>>      >     > which does exactly this.
>>>>      >
>>>>      >     > We window by the event time, but trigger punctuate in
>>> <punctuate interval>
>>>>      >
>>>>      >     > duration of system time, in the absence of an event
>>>> crossing
>>> the punctuate
>>>>      >
>>>>      >     > event time.
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > We are using Kafka Stream for our Audit Trail, where
>>>> we need
>>> to output the
>>>>      >
>>>>      >     > event counts on each topic on each cluster aggregated
>>>> over a
>>> 1 minute
>>>>      >
>>>>      >     > window. We have to use event time to be able to cross
>>>> check
>>> the counts. But
>>>>      >
>>>>      >     > we need to trigger punctuate [aggregate event pushes]
>>>> by
>>> system time in the
>>>>      >
>>>>      >     > absence of events. Otherwise the event counts for
>>>> unexpired
>>> windows would
>>>>      >
>>>>      >     > be 0 which is bad.
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > "Maybe a hybrid solution works: I window by event
>>>> time but
>>> trigger results
>>>>      >
>>>>      >     > by system time for windows that have updated? Not
>>>> really sure
>>> the details
>>>>      >
>>>>      >     > of making that work. Does that work? Are there
>>>> concrete
>>> examples where you
>>>>      >
>>>>      >     > actually want the current behavior?"
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > --
>>>>      >
>>>>      >     > With Regards,
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > Arun Mathew
>>>>      >
>>>>      >     > Yahoo! JAPAN Corporation
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>> skyahead@gmail.com>
>>> <sk...@gmail.com> wrote:
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >     >> Hi Jay,
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >> The hybrid solution is exactly what I expect and
>>>> need for
>>> our use cases
>>>>      >
>>>>      >     >> when dealing with telecom data.
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >> Thanks
>>>>      >
>>>>      >     >> Tianji
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>> jay@confluent.io>
>>> <ja...@confluent.io> wrote:
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >>> Hey guys,
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> One thing I've always found super important for
>>>> this kind
>>> of design work
>>>>      >
>>>>      >     >> is
>>>>      >
>>>>      >     >>> to do a really good job of cataloging the landscape
>>>> of use
>>> cases and how
>>>>      >
>>>>      >     >>> prevalent each one is. By that I mean not just
>>>> listing lots
>>> of uses, but
>>>>      >
>>>>      >     >>> also grouping them into categories that
>>>> functionally need
>>> the same thing.
>>>>      >
>>>>      >     >>> In the absence of this it is very hard to reason
>>>> about
>>> design proposals.
>>>>      >
>>>>      >     >>> From the proposals so far I think we have a lot of
>>> discussion around
>>>>      >
>>>>      >     >>> possible apis, but less around what the user needs
>>>> for
>>> different use
>>>>      >
>>>>      >     >> cases
>>>>      >
>>>>      >     >>> and how they would implement that using the api.
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> Here is an example:
>>>>      >
>>>>      >     >>> You aggregate click and impression data for a
>>>> reddit like
>>> site. Every ten
>>>>      >
>>>>      >     >>> minutes you want to output a ranked list of the top
>>>> 10
>>> articles ranked by
>>>>      >
>>>>      >     >>> clicks/impressions for each geographical area. I
>>>> want to be
>>> able run this
>>>>      >
>>>>      >     >>> in steady state as well as rerun to regenerate
>>>> results (or
>>> catch up if it
>>>>      >
>>>>      >     >>> crashes).
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> There are a couple of tricky things that seem to
>>>> make this
>>> hard with
>>>>      >
>>>>      >     >> either
>>>>      >
>>>>      >     >>> of the options proposed:
>>>>      >
>>>>      >     >>> 1. If I emit this data using event time I have the
>>>> problem
>>> described
>>>>      >
>>>>      >     >> where
>>>>      >
>>>>      >     >>> a geographical region with no new clicks or
>>>> impressions
>>> will fail to
>>>>      >
>>>>      >     >> output
>>>>      >
>>>>      >     >>> results.
>>>>      >
>>>>      >     >>> 2. If I emit this data using system time I have the
>>>> problem
>>> that when
>>>>      >
>>>>      >     >>> reprocessing data my window may not be ten minutes
>>>> but 10
>>> hours if my
>>>>      >
>>>>      >     >>> processing is very fast so it dramatically changes
>>>> the
>>> output.
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> Maybe a hybrid solution works: I window by event
>>>> time but
>>> trigger results
>>>>      >
>>>>      >     >>> by system time for windows that have updated? Not
>>>> really
>>> sure the details
>>>>      >
>>>>      >     >>> of making that work. Does that work? Are there
>>>> concrete
>>> examples where
>>>>      >
>>>>      >     >> you
>>>>      >
>>>>      >     >>> actually want the current behavior?
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> -Jay
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>> arunmathew88@gmail.com> <ar...@gmail.com>
>>>>      >
>>>>      >     >>> wrote:
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>>> Hi All,
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> Thanks for the KIP. We were also in need of a
>>>> mechanism to
>>> trigger
>>>>      >
>>>>      >     >>>> punctuate in the absence of events.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> As I described in [
>>>>      >
>>>>      >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>      >
>>>>      >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>      >
>>>>      >     >>>> plugin.system.issuetabpanels:comment-
>>>> tabpanel#comment-
>>> 15926036
>>>>      >
>>>>      >     >>>> ],
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>    - Our approached involved using the event time
>>>> by
>>> default.
>>>>      >
>>>>      >     >>>>    - The method to check if there is any punctuate
>>>> ready
>>> in the
>>>>      >
>>>>      >     >>>>    PunctuationQueue is triggered via the any event
>>> received by the
>>>>      >
>>>>      >     >> stream
>>>>      >
>>>>      >     >>>>    tread, or at the polling intervals in the
>>>> absence of
>>> any events.
>>>>      >
>>>>      >     >>>>    - When we create Punctuate objects (which
>>>> contains the
>>> next event
>>>>      >
>>>>      >     >> time
>>>>      >
>>>>      >     >>>>    for punctuation and interval), we also record
>>>> the
>>> creation time
>>>>      >
>>>>      >     >>> (system
>>>>      >
>>>>      >     >>>>    time).
>>>>      >
>>>>      >     >>>>    - While checking for maturity of Punctuate
>>>> Schedule by
>>>>      >
>>>>      >     >> mayBePunctuate
>>>>      >
>>>>      >     >>>>    method, we also check if the system clock has
>>>> elapsed
>>> the punctuate
>>>>      >
>>>>      >     >>>>    interval since the schedule creation time.
>>>>      >
>>>>      >     >>>>    - In the absence of any event, or in the
>>>> absence of any
>>> event for
>>>>      >
>>>>      >     >> one
>>>>      >
>>>>      >     >>>>    topic in the partition group assigned to the
>>>> stream
>>> task, the system
>>>>      >
>>>>      >     >>>> time
>>>>      >
>>>>      >     >>>>    will elapse the interval and we trigger a
>>>> punctuate
>>> using the
>>>>      >
>>>>      >     >> expected
>>>>      >
>>>>      >     >>>>    punctuation event time.
>>>>      >
>>>>      >     >>>>    - we then create the next punctuation schedule
>>>> as
>>> punctuation event
>>>>      >
>>>>      >     >>> time
>>>>      >
>>>>      >     >>>>    + punctuation interval, [again recording the
>>>> system
>>> time of creation
>>>>      >
>>>>      >     >>> of
>>>>      >
>>>>      >     >>>> the
>>>>      >
>>>>      >     >>>>    schedule].
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>> approach
>>> has pros and
>>>>      >
>>>>      >     >>>> cons.
>>>>      >
>>>>      >     >>>> Pros
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>    - Punctuates will happen in <punctuate
>>>> interval> time
>>> duration at
>>>>      >
>>>>      >     >> max
>>>>      >
>>>>      >     >>> in
>>>>      >
>>>>      >     >>>>    terms of system time.
>>>>      >
>>>>      >     >>>>    - The semantics as a whole continues to revolve
>>>> around
>>> event time.
>>>>      >
>>>>      >     >>>>    - We can use the old data [old timestamps] to
>>>> rerun any
>>> experiments
>>>>      >
>>>>      >     >> or
>>>>      >
>>>>      >     >>>>    tests.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> Cons
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>    - In case the  <punctuate interval> is not a
>>>> time
>>> duration [say
>>>>      >
>>>>      >     >>> logical
>>>>      >
>>>>      >     >>>>    time/event count], then the approach might not
>>>> be
>>> meaningful.
>>>>      >
>>>>      >     >>>>    - In case there is a case where we have to wait
>>>> for an
>>> actual event
>>>>      >
>>>>      >     >>> from
>>>>      >
>>>>      >     >>>>    a low event rate partition in the partition
>>>> group, this
>>> approach
>>>>      >
>>>>      >     >> will
>>>>      >
>>>>      >     >>>> jump
>>>>      >
>>>>      >     >>>>    the gun.
>>>>      >
>>>>      >     >>>>    - in case the event processing cannot catch up
>>>> with the
>>> event rate
>>>>      >
>>>>      >     >> and
>>>>      >
>>>>      >     >>>>    the expected timestamp events gets queued for
>>>> long
>>> time, this
>>>>      >
>>>>      >     >> approach
>>>>      >
>>>>      >     >>>>    might jump the gun.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> I believe the above approach and discussion goes
>>>> close to
>>> the approach
>>>>      >
>>>>      >     >> A.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> -----------
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> I like the idea of having an even count based
>>>> punctuate.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> -----------
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> I agree with the discussion around approach C,
>>>> that we
>>> should provide
>>>>      >
>>>>      >     >> the
>>>>      >
>>>>      >     >>>> user with the option to choose system time or
>>>> event time
>>> based
>>>>      >
>>>>      >     >>> punctuates.
>>>>      >
>>>>      >     >>>> But I believe that the user predominantly wants to
>>>> use
>>> event time while
>>>>      >
>>>>      >     >>> not
>>>>      >
>>>>      >     >>>> missing out on regular punctuates due to event
>>>> delays or
>>> event
>>>>      >
>>>>      >     >> absences.
>>>>      >
>>>>      >     >>>> Hence a complex punctuate option as Matthias
>>>> mentioned
>>> (quoted below)
>>>>      >
>>>>      >     >>> would
>>>>      >
>>>>      >     >>>> be most apt.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> "- We might want to add "complex" schedules later
>>>> on
>>> (like, punctuate
>>>>      >
>>>>      >     >> on
>>>>      >
>>>>      >     >>>> every 10 seconds event-time or 60 seconds system-
>>>> time
>>> whatever comes
>>>>      >
>>>>      >     >>>> first)."
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> -----------
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> I think I read somewhere that Kafka Streams
>>>> started with
>>> System Time as
>>>>      >
>>>>      >     >>> the
>>>>      >
>>>>      >     >>>> punctuation standard, but was later changed to
>>>> Event Time.
>>> I guess
>>>>      >
>>>>      >     >> there
>>>>      >
>>>>      >     >>>> would be some good reason behind it. As Kafka
>>>> Streams want
>>> to evolve
>>>>      >
>>>>      >     >> more
>>>>      >
>>>>      >     >>>> on the Stream Processing front, I believe the
>>>> emphasis on
>>> event time
>>>>      >
>>>>      >     >>> would
>>>>      >
>>>>      >     >>>> remain quite strong.
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> With Regards,
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> Arun Mathew
>>>>      >
>>>>      >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>> tobecker@tivo.com> <to...@tivo.com>
>>>>      >
>>>>      >     >> wrote:
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>>> Yeah I like PuncutationType much better; I just
>>>> threw
>>> Time out there
>>>>      >
>>>>      >     >>>>> more as a strawman than an actual suggestion ;) I
>>>> still
>>> think it's
>>>>      >
>>>>      >     >>>>> worth considering what this buys us over an
>>>> additional
>>> callback. I
>>>>      >
>>>>      >     >>>>> foresee a number of punctuate implementations
>>>> following
>>> this pattern:
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> public void punctuate(PunctuationType type) {
>>>>      >
>>>>      >     >>>>>     switch (type) {
>>>>      >
>>>>      >     >>>>>         case EVENT_TIME:
>>>>      >
>>>>      >     >>>>>             methodA();
>>>>      >
>>>>      >     >>>>>             break;
>>>>      >
>>>>      >     >>>>>         case SYSTEM_TIME:
>>>>      >
>>>>      >     >>>>>             methodB();
>>>>      >
>>>>      >     >>>>>             break;
>>>>      >
>>>>      >     >>>>>     }
>>>>      >
>>>>      >     >>>>> }
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> I guess one advantage of this approach is we
>>>> could add
>>> additional
>>>>      >
>>>>      >     >>>>> punctuation types later in a backwards compatible
>>>> way
>>> (like event
>>>>      >
>>>>      >     >> count
>>>>      >
>>>>      >     >>>>> as you mentioned).
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> -Tommy
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>> Sax wrote:
>>>>      >
>>>>      >     >>>>>> That sounds promising.
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> I am just wondering if `Time` is the best name.
>>>> Maybe we
>>> want to
>>>>      >
>>>>      >     >> add
>>>>      >
>>>>      >     >>>>>> other non-time based punctuations at some point
>>>> later. I
>>> would
>>>>      >
>>>>      >     >>>>>> suggest
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> enum PunctuationType {
>>>>      >
>>>>      >     >>>>>>   EVENT_TIME,
>>>>      >
>>>>      >     >>>>>>   SYSTEM_TIME,
>>>>      >
>>>>      >     >>>>>> }
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> or similar. Just to keep the door open -- it's
>>>> easier to
>>> add new
>>>>      >
>>>>      >     >>>>>> stuff
>>>>      >
>>>>      >     >>>>>> if the name is more generic.
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> -Matthias
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>>
>>>>      >
>>>>      >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> I agree that the framework providing and
>>>> managing the
>>> notion of
>>>>      >
>>>>      >     >>>>>>> stream
>>>>      >
>>>>      >     >>>>>>> time is valuable and not something we would
>>>> want to
>>> delegate to
>>>>      >
>>>>      >     >> the
>>>>      >
>>>>      >     >>>>>>> tasks. I'm not entirely convinced that a
>>>> separate
>>> callback
>>>>      >
>>>>      >     >> (option
>>>>      >
>>>>      >     >>>>>>> C)
>>>>      >
>>>>      >     >>>>>>> is that messy (it could just be a default
>>>> method with
>>> an empty
>>>>      >
>>>>      >     >>>>>>> implementation), but if we wanted a single API
>>>> to
>>> handle both
>>>>      >
>>>>      >     >>>>>>> cases,
>>>>      >
>>>>      >     >>>>>>> how about something like the following?
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> enum Time {
>>>>      >
>>>>      >     >>>>>>>    STREAM,
>>>>      >
>>>>      >     >>>>>>>    CLOCK
>>>>      >
>>>>      >     >>>>>>> }
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> Then on ProcessorContext:
>>>>      >
>>>>      >     >>>>>>> context.schedule(Time time, long interval)  //
>>>> We could
>>> allow
>>>>      >
>>>>      >     >> this
>>>>      >
>>>>      >     >>>>>>> to
>>>>      >
>>>>      >     >>>>>>> be called once for each value of time to mix
>>> approaches.
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> Then the Processor API becomes:
>>>>      >
>>>>      >     >>>>>>> punctuate(Time time) // time here denotes which
>>> schedule resulted
>>>>      >
>>>>      >     >>>>>>> in
>>>>      >
>>>>      >     >>>>>>> this call.
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> Thoughts?
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>> Sax
>>> wrote:
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> I was thinking about the four options you
>>>> proposed in
>>> more
>>>>      >
>>>>      >     >>>>>>>> details
>>>>      >
>>>>      >     >>>>>>>> and
>>>>      >
>>>>      >     >>>>>>>> this are my thoughts:
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> (A) You argue, that users can still
>>>> "punctuate" on
>>> event-time
>>>>      >
>>>>      >     >> via
>>>>      >
>>>>      >     >>>>>>>> process(), but I am not sure if this is
>>>> possible.
>>> Note, that
>>>>      >
>>>>      >     >>>>>>>> users
>>>>      >
>>>>      >     >>>>>>>> only
>>>>      >
>>>>      >     >>>>>>>> get record timestamps via context.timestamp().
>>>> Thus,
>>> users
>>>>      >
>>>>      >     >> would
>>>>      >
>>>>      >     >>>>>>>> need
>>>>      >
>>>>      >     >>>>>>>> to
>>>>      >
>>>>      >     >>>>>>>> track the time progress per partition (based
>>>> on the
>>> partitions
>>>>      >
>>>>      >     >>>>>>>> they
>>>>      >
>>>>      >     >>>>>>>> obverse via context.partition(). (This alone
>>>> puts a
>>> huge burden
>>>>      >
>>>>      >     >>>>>>>> on
>>>>      >
>>>>      >     >>>>>>>> the
>>>>      >
>>>>      >     >>>>>>>> user by itself.) However, users are not
>>>> notified at
>>> startup
>>>>      >
>>>>      >     >> what
>>>>      >
>>>>      >     >>>>>>>> partitions are assigned, and user are not
>>>> notified
>>> when
>>>>      >
>>>>      >     >>>>>>>> partitions
>>>>      >
>>>>      >     >>>>>>>> get
>>>>      >
>>>>      >     >>>>>>>> revoked. Because this information is not
>>>> available,
>>> it's not
>>>>      >
>>>>      >     >>>>>>>> possible
>>>>      >
>>>>      >     >>>>>>>> to
>>>>      >
>>>>      >     >>>>>>>> "manually advance" stream-time, and thus
>>>> event-time
>>> punctuation
>>>>      >
>>>>      >     >>>>>>>> within
>>>>      >
>>>>      >     >>>>>>>> process() seems not to be possible -- or do
>>>> you see a
>>> way to
>>>>      >
>>>>      >     >> get
>>>>      >
>>>>      >     >>>>>>>> it
>>>>      >
>>>>      >     >>>>>>>> done? And even if, it might still be too
>>>> clumsy to
>>> use.
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> (B) This does not allow to mix both
>>>> approaches, thus
>>> limiting
>>>>      >
>>>>      >     >>>>>>>> what
>>>>      >
>>>>      >     >>>>>>>> users
>>>>      >
>>>>      >     >>>>>>>> can do.
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> (C) This should give all flexibility we need.
>>>> However,
>>> just
>>>>      >
>>>>      >     >>>>>>>> adding
>>>>      >
>>>>      >     >>>>>>>> one
>>>>      >
>>>>      >     >>>>>>>> more method seems to be a solution that is too
>>>> simple
>>> (cf my
>>>>      >
>>>>      >     >>>>>>>> comments
>>>>      >
>>>>      >     >>>>>>>> below).
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>> sure how
>>> a user
>>>>      >
>>>>      >     >>>>>>>> could
>>>>      >
>>>>      >     >>>>>>>> enable system-time and event-time punctuation
>>>> in
>>> parallel.
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> Overall options (C) seems to be the most
>>>> promising
>>> approach to
>>>>      >
>>>>      >     >>>>>>>> me.
>>>>      >
>>>>      >     >>>>>>>> Because I also favor a clean API, we might
>>>> keep
>>> current
>>>>      >
>>>>      >     >>>>>>>> punctuate()
>>>>      >
>>>>      >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>> at some
>>> later
>>>>      >
>>>>      >     >>>>>>>> point
>>>>      >
>>>>      >     >>>>>>>> when
>>>>      >
>>>>      >     >>>>>>>> people use the "new punctuate API".
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> Couple of follow up questions:
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - I am wondering, if we should have two
>>>> callback
>>> methods or
>>>>      >
>>>>      >     >> just
>>>>      >
>>>>      >     >>>>>>>> one
>>>>      >
>>>>      >     >>>>>>>> (ie, a unified for system and event time
>>>> punctuation
>>> or one for
>>>>      >
>>>>      >     >>>>>>>> each?).
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - If we have one, how can the user figure out,
>>>> which
>>> condition
>>>>      >
>>>>      >     >>>>>>>> did
>>>>      >
>>>>      >     >>>>>>>> trigger?
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - How would the API look like, for registering
>>> different
>>>>      >
>>>>      >     >>>>>>>> punctuate
>>>>      >
>>>>      >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - We might want to add "complex" schedules
>>>> later on
>>> (like,
>>>>      >
>>>>      >     >>>>>>>> punctuate
>>>>      >
>>>>      >     >>>>>>>> on
>>>>      >
>>>>      >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>> system-time
>>> whatever
>>>>      >
>>>>      >     >>>>>>>> comes
>>>>      >
>>>>      >     >>>>>>>> first). I don't say we should add this right
>>>> away, but
>>> we might
>>>>      >
>>>>      >     >>>>>>>> want
>>>>      >
>>>>      >     >>>>>>>> to
>>>>      >
>>>>      >     >>>>>>>> define the API in a way, that it allows
>>>> extensions
>>> like this
>>>>      >
>>>>      >     >>>>>>>> later
>>>>      >
>>>>      >     >>>>>>>> on,
>>>>      >
>>>>      >     >>>>>>>> without redesigning the API (ie, the API
>>>> should be
>>> designed
>>>>      >
>>>>      >     >>>>>>>> extensible)
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> - Did you ever consider count-based
>>>> punctuation?
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> I understand, that you would like to solve a
>>>> simple
>>> problem,
>>>>      >
>>>>      >     >> but
>>>>      >
>>>>      >     >>>>>>>> we
>>>>      >
>>>>      >     >>>>>>>> learned from the past, that just "adding some
>>>> API"
>>> quickly
>>>>      >
>>>>      >     >> leads
>>>>      >
>>>>      >     >>>>>>>> to a
>>>>      >
>>>>      >     >>>>>>>> not very well defined API that needs time
>>>> consuming
>>> clean up
>>>>      >
>>>>      >     >>>>>>>> later on
>>>>      >
>>>>      >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>> holistic
>>>>      >
>>>>      >     >>>>>>>> punctuation
>>>>      >
>>>>      >     >>>>>>>> KIP
>>>>      >
>>>>      >     >>>>>>>> with this from the beginning on to avoid later
>>>> painful
>>>>      >
>>>>      >     >> redesign.
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> -Matthias
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>
>>>>      >
>>>>      >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> Thanks Thomas,
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> I'm also wary of changing the existing
>>>> semantics of
>>>>      >
>>>>      >     >> punctuate,
>>>>      >
>>>>      >     >>>>>>>>> for
>>>>      >
>>>>      >     >>>>>>>>> backward compatibility reasons, although I
>>>> like the
>>>>      >
>>>>      >     >> conceptual
>>>>      >
>>>>      >     >>>>>>>>> simplicity of that option.
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>> a way,
>>> uglier.
>>>>      >
>>>>      >     >> I
>>>>      >
>>>>      >     >>>>>>>>> added
>>>>      >
>>>>      >     >>>>>>>>> this to the KIP now as option (C).
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>> more
>>> flexible,
>>>>      >
>>>>      >     >> as
>>>>      >
>>>>      >     >>>>>>>>> it
>>>>      >
>>>>      >     >>>>>>>>> allows
>>>>      >
>>>>      >     >>>>>>>>> you to return any value, you're not limited
>>>> to event
>>> time or
>>>>      >
>>>>      >     >>>>>>>>> system
>>>>      >
>>>>      >     >>>>>>>>> time
>>>>      >
>>>>      >     >>>>>>>>> (although I don't see an actual use case
>>>> where you
>>> might need
>>>>      >
>>>>      >     >>>>>>>>> anything
>>>>      >
>>>>      >     >>>>>>>>> else then those two). Hence I also proposed
>>>> the
>>> option to
>>>>      >
>>>>      >     >> allow
>>>>      >
>>>>      >     >>>>>>>>> users
>>>>      >
>>>>      >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>> for
>>> them given
>>>>      >
>>>>      >     >>>>>>>>> the
>>>>      >
>>>>      >     >>>>>>>>> presence or absence of messages, much like
>>>> they can
>>> decide
>>>>      >
>>>>      >     >> what
>>>>      >
>>>>      >     >>>>>>>>> msg
>>>>      >
>>>>      >     >>>>>>>>> time
>>>>      >
>>>>      >     >>>>>>>>> means for them using the TimestampExtractor.
>>>> What do
>>> you
>>>>      >
>>>>      >     >> think
>>>>      >
>>>>      >     >>>>>>>>> about
>>>>      >
>>>>      >     >>>>>>>>> that? This is probably most flexible but also
>>>> most
>>>>      >
>>>>      >     >> complicated.
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> All comments appreciated.
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> Cheers,
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> Michal
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>      >
>>>>      >     >>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>> Although I fully agree we need a way to
>>>> trigger
>>> periodic
>>>>      >
>>>>      >     >>>>>>>>>> processing
>>>>      >
>>>>      >     >>>>>>>>>> that is independent from whether and when
>>>> messages
>>> arrive,
>>>>      >
>>>>      >     >>>>>>>>>> I'm
>>>>      >
>>>>      >     >>>>>>>>>> not sure
>>>>      >
>>>>      >     >>>>>>>>>> I like the idea of changing the existing
>>>> semantics
>>> across
>>>>      >
>>>>      >     >> the
>>>>      >
>>>>      >     >>>>>>>>>> board.
>>>>      >
>>>>      >     >>>>>>>>>> What if we added an additional callback to
>>>> Processor
>>> that
>>>>      >
>>>>      >     >> can
>>>>      >
>>>>      >     >>>>>>>>>> be
>>>>      >
>>>>      >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>> always
>>> called at
>>>>      >
>>>>      >     >>>>>>>>>> fixed, wall
>>>>      >
>>>>      >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>> have to
>>> give
>>>>      >
>>>>      >     >> up
>>>>      >
>>>>      >     >>>>>>>>>> the
>>>>      >
>>>>      >     >>>>>>>>>> notion
>>>>      >
>>>>      >     >>>>>>>>>> of stream time to be able to do periodic
>>>> processing.
>>>>      >
>>>>      >     >>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>> Borowiecki
>>> wrote:
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> Hi all,
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>> punctuate
>>>>      >
>>>>      >     >>>>>>>>>>> semantics
>>>>      >
>>>>      >     >>>>>>>>>>> <https://cwiki.apache.org/
>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
>>> confluence/display/KAFKA/KIP->
>>>>      >
>>>>      >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI
>>>> P->>>
>>> 138%
>>>>      >
>>>>      >     >>>>>>>>>>> 3A+C
>>>>      >
>>>>      >     >>>>>>>>>>> hange+
>>>>      >
>>>>      >     >>>>>>>>>>> punctuate+semantics>
>>>>      >
>>>>      >     >>>>>>>>>>> .
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> Appreciating there can be different views
>>>> on
>>> system-time
>>>>      >
>>>>      >     >> vs
>>>>      >
>>>>      >     >>>>>>>>>>> event-
>>>>      >
>>>>      >     >>>>>>>>>>> time
>>>>      >
>>>>      >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>> case and
>>> the
>>>>      >
>>>>      >     >>>>>>>>>>> importance of
>>>>      >
>>>>      >     >>>>>>>>>>> backwards compatibility of any such change,
>>>> I've
>>> left it
>>>>      >
>>>>      >     >>>>>>>>>>> quite
>>>>      >
>>>>      >     >>>>>>>>>>> open
>>>>      >
>>>>      >     >>>>>>>>>>> and
>>>>      >
>>>>      >     >>>>>>>>>>> hope to fill in more info as the discussion
>>> progresses.
>>>>      >
>>>>      >     >>>>>>>>>>>
>>>>      >
>>>>      >     >>>>>>>>>>> Thanks,
>>>>      >
>>>>      >     >>>>>>>>>>> Michal
>>>>      >
>>>>      >     >>>>>>> --
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>     Tommy Becker
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>     Senior Software Engineer
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>     tivo.com
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> ________________________________
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>>>> This email and any attachments may contain
>>>> confidential
>>> and
>>>>      >
>>>>      >     >>>>>>> privileged material for the sole use of the
>>>> intended
>>> recipient.
>>>>      >
>>>>      >     >> Any
>>>>      >
>>>>      >     >>>>>>> review, copying, or distribution of this email
>>>> (or any
>>>>      >
>>>>      >     >> attachments)
>>>>      >
>>>>      >     >>>>>>> by others is prohibited. If you are not the
>>>> intended
>>> recipient,
>>>>      >
>>>>      >     >>>>>>> please contact the sender immediately and
>>>> permanently
>>> delete this
>>>>      >
>>>>      >     >>>>>>> email and any attachments. No employee or agent
>>>> of TiVo
>>> Inc. is
>>>>      >
>>>>      >     >>>>>>> authorized to conclude any binding agreement on
>>>> behalf
>>> of TiVo
>>>>      >
>>>>      >     >> Inc.
>>>>      >
>>>>      >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>> only be
>>> made by a
>>>>      >
>>>>      >     >>>>>>> signed written agreement.
>>>>      >
>>>>      >     >>>>>>>
>>>>      >
>>>>      >     >>>>> --
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>     Tommy Becker
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>     Senior Software Engineer
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>     tivo.com
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> ________________________________
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>> This email and any attachments may contain
>>>> confidential
>>> and
>>>>      >
>>>>      >     >> privileged
>>>>      >
>>>>      >     >>>>> material for the sole use of the intended
>>>> recipient. Any
>>> review,
>>>>      >
>>>>      >     >>> copying,
>>>>      >
>>>>      >     >>>>> or distribution of this email (or any
>>>> attachments) by
>>> others is
>>>>      >
>>>>      >     >>>> prohibited.
>>>>      >
>>>>      >     >>>>> If you are not the intended recipient, please
>>>> contact the
>>> sender
>>>>      >
>>>>      >     >>>>> immediately and permanently delete this email and
>>>> any
>>> attachments. No
>>>>      >
>>>>      >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>> conclude
>>> any binding
>>>>      >
>>>>      >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>> Binding
>>> agreements with
>>>>      >
>>>>      >     >> TiVo
>>>>      >
>>>>      >     >>>>> Inc. may only be made by a signed written
>>>> agreement.
>>>>      >
>>>>      >     >>>>>
>>>>      >
>>>>      >     >>>>
>>>>      >
>>>>      >     >>>
>>>>      >
>>>>      >     >>
>>>>      >
>>>>      >     >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      >
>>>>      > --
>>>>      >
>>>>      > <http://www.openbet.com/>
>>>>      >
>>>>      > *Michal Borowiecki*
>>>>      >
>>>>      > *Senior Software Engineer L4*
>>>>      >
>>>>      > *T: *
>>>>      >
>>>>      > +44 208 742 1600 <+44%2020%208742%201600>
>>>>      >
>>>>      > +44 203 249 8448 <+44%2020%203249%208448>
>>>>      >
>>>>      >
>>>>      >
>>>>      > *E: *
>>>>      >
>>>>      > michal.borowiecki@openbet.com
>>>>      >
>>>>      > *W: *
>>>>      >
>>>>      > www.openbet.com
>>>>      >
>>>>      > *OpenBet Ltd*
>>>>      >
>>>>      > Chiswick Park Building 9
>>>>      >
>>>>      > 566 Chiswick High Rd
>>>>      >
>>>>      > London
>>>>      >
>>>>      > W4 5XT
>>>>      >
>>>>      > UK
>>>>      >
>>>>      > <https://www.openbet.com/email_promo>
>>>>      >
>>>>      >
>>>>      >
>>>>      > This message is confidential and intended only for the
>>>> addressee.
>>> If you
>>>>      > have received this message in error, please immediately
>>>> notify the
>>>>      > postmaster@openbet.com and delete it from your system as
>>>> well as
>>> any
>>>>      > copies. The content of e-mails as well as traffic data may
>>>> be
>>> monitored by
>>>>      > OpenBet for employment and security purposes. To protect
>>>> the
>>> environment
>>>>      > please do not print this e-mail unless necessary. OpenBet
>>>> Ltd.
>>> Registered
>>>>      > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>> London,
>>> W4 5XT,
>>>>      > United Kingdom. A company registered in England and Wales.
>>> Registered no.
>>>>      > 3134634. VAT no. GB927523612
>>>>      >
>>>>      >
>>>>      >
>>>>
>>>>
> --
>
>
>      Tommy Becker
>
>      Senior Software Engineer
>
>      O +1 919.460.4747
>
>      tivo.com
>
>
> ________________________________
>
> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Hi,

I would like to push this discussion further. Thanks Michael for your
email (it was longer than expected, even after your "sorry for the long
email" warning... :P). But I'll go ahead an follow your example: it's a
long read. :)

I think it's really important to put all this in a larger scope and
think Michael's comments are super useful here.

Kafka Streams DSL does not have triggers (for a good reason, IMHO) and
there are no plans to add them at the moment. IMHO, the whole
Dataflow/Beam style processing approach, mingles together two different
things into one big and hard to use API. A clean DSL, should describe
what the result should be but not more -- everything else, "pollutes"
the code.

I understand the desire to have fine grained control of all kind of
things -- but I also believe that fine grained control belongs to
Processor API -- not to the DSL (at least not at this level). Therefore,
from my point of view, whatever punctuation mechanism we introduce, it
should not be part of the DSL (note, you can of course mix-and-match
high-level DSL and low-level PAPI).

If there is need, to give better control at the DSL level, we can of
course design something -- but it should not be punctuations or changes
to the DSL itself. To me, this is more at the "configuration level".

I want to pick up the example of KIP-63: it added a record cache to
reduce the downstream load. This is a pure "performance optimization".
Note, that the output of an aggregation is a changelog stream -- thus,
"deduplication" does not change the result. It only holds back certain
updates, and if new updates are coming in while holding back, you don't
see some "intermediate results" -- but you don't need to see those, as
they are not "correct" anyway. I point this out to  it contrast it to
triggers, what have a completely different purpose.

Furthermore, I personally believe, that __change__ is the next step in
the stream processing world -- and Kafka Stream embraces change ("Yes,
we can" -- couldn't resist...). Thus, (IMHO) application should be
designed with change in mind -- in the Stream processing world, there is
nothing like a final result -- that is the "lie" the Dataflow/Beam model
tells you. Of course, if you need to make a decision in real life based
in some data, you very often cannot undo a decision and also cannot wait
forever to decide. But you need to do this decision based on the
"current data" (I don't call is "final result" on purpose as there is no
such thing -- or to put it differently: it might exist, but you can
never now if it is final, as there might always be another late arriving
record). With this regard, wall-clock time punctuation are very useful
(and also IQ, that allows you to lock up the __current__ state).

Having said this, it's of course impossible to preserve the whole
history -- and also not required. As some point, late arriving data are
not interesting anymore -- maybe somebody made a decision and the world
move on (late data might only show you that the decision was wrong, but
it would be too late to correct it). For this reason, Streams has the
notion of "retention time". With this regard, you can argue that you get
a "final result" after retention time passed. But again, it not part of
the "logical description" of the result that is specified by the DSL --
it's an operation concern concern.

I strongly believe, that this overall design of the Streams DSL is
important to put into account. I know, this is more a "meta" email, and
does not give a detailed answer about the punctuation discussion (or not
much -- maybe: "not part of DSL"). But it should help to put the use
cases we collect into the right buckets, and maybe also help to identify
what we need to improve on the DSL to improve its usability, as a fall
back to PAPI is always cumbersome.


Looking forward to your feedback.


-Matthias






On 4/11/17 12:52 PM, Thomas Becker wrote:
> Here's an example that we currently have.  We have a streams processor
> that does a transform from one topic into another. One of the fields in
> the source topic record is an expiration time, and one of the functions
> of the processor is to ensure that expired records get deleted promptly
> after that time passes (typically days or weeks after the message was
> originally produced). To do that, the processor keeps a state store of
> keys and expiration times, iterates that store in punctuate(), and
> emits delete (null) records for expired items. This needs to happen at
> some minimum interval regardless of the incoming message rate of the
> source topic.
> 
> In this scenario, the expiration of records is the primary function of
> punctuate, and therefore the key requirement is that the wall-clock
> measured time between punctuate calls have some upper-bound. So a pure
> wall-clock based schedule would be fine for our needs. But the proposed
> "hybrid" system would also be acceptable if that satisfies a broader
> range of use-cases.
> 
> On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
>> I apologize for the longer email below.  To my defense, it started
>> out much
>> shorter. :-)  Also, to be super-clear, I am intentionally playing
>> devil's
>> advocate for a number of arguments brought forth in order to help
>> improve
>> this KIP -- I am not implying I necessarily disagree with the
>> arguments.
>>
>> That aside, here are some further thoughts.
>>
>> First, there are (at least?) two categories for actions/behavior you
>> invoke
>> via punctuate():
>>
>> 1. For internal housekeeping of your Processor or Transformer (e.g.,
>> to
>> periodically commit to a custom store, to do metrics/logging).  Here,
>> the
>> impact of punctuate is typically not observable by other processing
>> nodes
>> in the topology.
>> 2. For controlling the emit frequency of downstream records.  Here,
>> the
>> punctuate is all about being observable by downstream processing
>> nodes.
>>
>> A few releases back, we introduced record caches (DSL) and state
>> store
>> caches (Processor API) in KIP-63.  Here, we addressed a concern
>> relating to
>> (2) where some users needed to control -- here: limit -- the
>> downstream
>> output rate of Kafka Streams because the downstream systems/apps
>> would not
>> be able to keep up with the upstream output rate (Kafka scalability >
>> their
>> scalability).  The argument for KIP-63, which notably did not
>> introduce a
>> "trigger" API, was that such an interaction with downstream systems
>> is an
>> operational concern;  it should not impact the processing *logic* of
>> your
>> application, and thus we didn't want to complicate the Kafka Streams
>> API,
>> especially not the declarative DSL, with such operational concerns.
>>
>> This KIP's discussion on `punctuate()` takes us back in time (<--
>> sorry, I
>> couldn't resist to not make this pun :-P).  As a meta-comment, I am
>> observing that our conversation is moving more and more into the
>> direction
>> of explicit "triggers" because, so far, I have seen only motivations
>> for
>> use cases in category (2), but none yet for (1)?  For example, some
>> comments voiced here are about sth like "IF stream-time didn't
>> trigger
>> punctuate, THEN trigger punctuate based on processing-time".  Do we
>> want
>> this, and if so, for which use cases and benefits?  Also, on a
>> related
>> note, whatever we are discussing here will impact state store caches
>> (Processor API) and perhaps also impact record caches (DSL), thus we
>> should
>> clarify any such impact here.
>>
>> Switching topics slightly.
>>
>> Jay wrote:
>>>
>>> One thing I've always found super important for this kind of design
>>> work
>>> is to do a really good job of cataloging the landscape of use cases
>>> and
>>> how prevalent each one is.
>> +1 to this, as others have already said.
>>
>> Here, let me highlight -- just in case -- that when we talked about
>> windowing use cases in the recent emails, the Processor API (where
>> `punctuate` resides) does not have any notion of windowing at
>> all.  If you
>> want to do windowing *in the Processor API*, you must do so manually
>> in
>> combination with window stores.  For this reason I'd suggest to
>> discuss use
>> cases not just in general, but also in view of how you'd do so in the
>> Processor API vs. in the DSL.  Right now, changing/improving
>> `punctuate`
>> does not impact the DSL at all, unless we add new functionality to
>> it.
>>
>> Jay wrote in his strawman example:
>>>
>>> You aggregate click and impression data for a reddit like site.
>>> Every ten
>>> minutes you want to output a ranked list of the top 10 articles
>>> ranked by
>>> clicks/impressions for each geographical area. I want to be able
>>> run this
>>> in steady state as well as rerun to regenerate results (or catch up
>>> if it
>>> crashes).
>> This is a good example for more than the obvious reason:  In KIP-63,
>> we
>> argued that the reason for saying "every ten minutes" above is not
>> necessarily about because you want to output data *exactly* after ten
>> minutes, but that you want to perform an aggregation based on 10-
>> minute
>> windows of input data; i.e., the point is about specifying the input
>> for
>> your aggregation, not or less about when the results of the
>> aggregation
>> should be send downstream.  To take an extreme example, you could
>> disable
>> record caches and let your app output a downstream update for every
>> incoming input record.  If the last input record was from at minute 7
>> of 10
>> (for a 10-min window), then what your app would output at minute 10
>> would
>> be identical to what it had already emitted at minute 7 earlier
>> anyways.
>> This is particularly true when we take late-arriving data into
>> account:  if
>> a late record arrived at minute 13, your app would (by default) send
>> a new
>> update downstream, even though the "original" 10 minutes have already
>> passed.
>>
>> Jay wrote...:
>>>
>>> There are a couple of tricky things that seem to make this hard
>>> with
>> either
>>>
>>> of the options proposed:
>>> 1. If I emit this data using event time I have the problem
>>> described where
>>> a geographical region with no new clicks or impressions will fail
>>> to
>> output
>>>
>>> results.
>> ...and Arun Mathew wrote:
>>
>>>
>>> We window by the event time, but trigger punctuate in <punctuate
>>> interval>
>>> duration of system time, in the absence of an event crossing the
>>> punctuate
>>> event time.
>> So, given what I wrote above about the status quo and what you can
>> already
>> do with it, is the concern that the state store cache doesn't give
>> you
>> *direct* control over "forcing an output after no later than X
>> seconds [of
>> processing-time]" but only indirect control through a cache
>> size?  (Note
>> that I am not dismissing the claims why this might be helpful.)
>>
>> Arun Mathew wrote:
>>>
>>> We are using Kafka Stream for our Audit Trail, where we need to
>>> output the
>>> event counts on each topic on each cluster aggregated over a 1
>>> minute
>>> window. We have to use event time to be able to cross check the
>>> counts.
>> But
>>>
>>> we need to trigger punctuate [aggregate event pushes] by system
>>> time in
>> the
>>>
>>> absence of events. Otherwise the event counts for unexpired windows
>>> would
>>> be 0 which is bad.
>> Isn't the latter -- "count would be 0" -- the problem between the
>> absence
>> of output vs. an output of 0, similar to the use of `Option[T]` in
>> Scala
>> and the difference between `None` and `Some(0)`?  That is, isn't the
>> root
>> cause that the downstream system interprets the absence of output in
>> a
>> particular way ("No output after 1 minute = I consider the output to
>> be
>> 0.")?  Arguably, you could also adapt the downstream system (if
>> possible)
>> to correctly handle the difference between absence of output vs.
>> output of
>> 0.  I am not implying that we shouldn't care about such a use case,
>> but
>> want to understand the motivation better. :-)
>>
>> Also, to add some perspective, in some related discussions we talked
>> about
>> how a Kafka Streams application should not worry or not be coupled
>> unnecessarily with such interpretation specifics in a downstream
>> system's
>> behavior.  After all, tomorrow your app's output might be consumed by
>> more
>> than just this one downstream system.  Arguably, Kafka Connect rather
>> than
>> Kafka Streams might be the best tool to link the universes of Kafka
>> and
>> downstream systems, including helping to reconcile the differences in
>> how
>> these systems interpret changes, updates, late-arriving data,
>> etc.  Kafka
>> Connect would allow you to decouple the Kafka Streams app's logical
>> processing from the specifics of downstream systems, thanks to
>> specific
>> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
>> this
>> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
>> but
>> it's currently awkward to use Connect for this", this might be a
>> problem we
>> can solve, too.)
>>
>> Switching topics slightly again.
>>
>> Thomas wrote:
>>>
>>> I'm not entirely convinced that a separate callback (option C)
>>> is that messy (it could just be a default method with an empty
>>> implementation), but if we wanted a single API to handle both
>>> cases,
>>> how about something like the following?
>>>
>>> enum Time {
>>>    STREAM,
>>>    CLOCK
>>> }
>> Yeah, I am on the fence here, too.  If we use the 1-method approach,
>> then
>> whatever the user is doing inside this method is a black box to Kafka
>> Streams (similar to how we have no idea what the user does inside a
>> `foreach` -- if the function passed to `foreach` writes to external
>> systems, then Kafka Streams is totally unaware of the fact).  We
>> won't
>> know, for example, if the stream-time action has a smaller "trigger"
>> frequency than the processing-time action.  Or, we won't know whether
>> the
>> user custom-codes a "not later than" trigger logic ("Do X every 1-
>> minute of
>> stream-time or 1-minute of processing-time, whichever comes
>> first").  That
>> said, I am not certain yet whether we would need such knowledge
>> because,
>> when using the Processor API, most of the work and decisions must be
>> done
>> by the user anyways.  It would matter though if the concept of
>> "triggers"
>> were to bubble up into the DSL because in the DSL the management of
>> windowing, window stores, etc. must be done automatically by Kafka
>> Streams.
>>
>> [In any case, btw, we have the corner case where the user configured
>> the
>> stream-time to be processing-time (e.g. via wall-clock timestamp
>> extractor), at which point both punctuate variants are based on the
>> same
>> time semantics / timeline.]
>>
>> Again, I apologize for the wall of text.  Congratulations if you made
>> it
>> this far. :-)
>>
>> More than happy to hear your thoughts!
>> Michael
>>
>> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com>
>> wrote:
>>
>>>
>>> Thanks Matthias.
>>> Sure, will correct it right away.
>>>
>>> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io>
>>> wrote:
>>>
>>> Thanks for preparing this page!
>>>
>>> About terminology:
>>>
>>> You introduce the term "event time" -- but we should call this
>>> "stream
>>> time" -- "stream time" is whatever TimestampExtractor returns and
>>> this
>>> could be event time, ingestion time, or processing/wall-clock time.
>>>
>>> Does this make sense to you?
>>>
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 4/10/17 4:58 AM, Arun Mathew wrote:
>>>>
>>>> Thanks Ewen.
>>>>
>>>> @Michal, @all, I have created a child page to start the Use Cases
>>> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
>>> Punctuate+Use+Cases]. Please go through it and give your comments.
>>>>
>>>>
>>>> @Tianji, Sorry for the delay. I am trying to make the patch
>>>> public.
>>>>
>>>> --
>>>> Arun Mathew
>>>>
>>>> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io>
>>>> wrote:
>>>>
>>>>     Arun,
>>>>
>>>>     I've given you permission to edit the wiki. Let me know if
>>>> you run
>>> into any
>>>>
>>>>     issues.
>>>>
>>>>     -Ewen
>>>>
>>>>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co
>>>> rp.jp>
>>> wrote:
>>>>
>>>>
>>>>     > Thanks Michal. I don’t have the access yet [arunmathew88].
>>>> Should I
>>> be
>>>>
>>>>     > sending a separate mail for this?
>>>>     >
>>>>     > I thought one of the person following this thread would be
>>>> able to
>>> give me
>>>>
>>>>     > access.
>>>>     >
>>>>     >
>>>>     >
>>>>     > *From: *Michal Borowiecki <mi...@openbet.com>
>>>>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>>>>     > *Date: *Friday, April 7, 2017 at 17:16
>>>>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>>>>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
>>>> semantics
>>>>     >
>>>>     >
>>>>     >
>>>>     > Hi Arun,
>>>>     >
>>>>     > I was thinking along the same lines as you, listing the use
>>>> cases
>>> on the
>>>>
>>>>     > wiki, but didn't find time to get around doing that yet.
>>>>     > Don't mind if you do it if you have access now.
>>>>     > I was thinking it would be nice if, once we have the use
>>>> cases
>>> listed,
>>>>
>>>>     > people could use likes to up-vote the use cases similar to
>>>> what
>>> they're
>>>>
>>>>     > working on.
>>>>     >
>>>>     > I should have a bit more time to action this in the next
>>>> few days,
>>> but
>>>>
>>>>     > happy for you to do it if you can beat me to it ;-)
>>>>     >
>>>>     > Cheers,
>>>>     > Michal
>>>>     >
>>>>     > On 07/04/17 04:39, Arun Mathew wrote:
>>>>     >
>>>>     > Sure, Thanks Matthias. My id is [arunmathew88].
>>>>     >
>>>>     >
>>>>     >
>>>>     > Of course. I was thinking of a subpage where people can
>>> collaborate.
>>>>
>>>>     >
>>>>     >
>>>>     >
>>>>     > Will do as per Michael’s suggestion.
>>>>     >
>>>>     >
>>>>     >
>>>>     > Regards,
>>>>     >
>>>>     > Arun Mathew
>>>>     >
>>>>     >
>>>>     >
>>>>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io>
>>>> <
>>> matthias@confluent.io> wrote:
>>>>
>>>>     >
>>>>     >
>>>>     >
>>>>     >     Please share your Wiki-ID and a committer can give you
>>>> write
>>> access.
>>>>
>>>>     >
>>>>     >
>>>>     >
>>>>     >     Btw: as you did not initiate the KIP, you should not
>>>> change the
>>> KIP
>>>>
>>>>     >
>>>>     >     without the permission of the original author -- in
>>>> this case
>>> Michael.
>>>>
>>>>     >
>>>>     >
>>>>     >
>>>>     >     So you might also just share your thought over the
>>>> mailing list
>>> and
>>>>
>>>>     >
>>>>     >     Michael can update the KIP page. Or, as an alternative,
>>>> just
>>> create a
>>>>
>>>>     >
>>>>     >     subpage for the KIP page.
>>>>     >
>>>>     >
>>>>     >
>>>>     >     @Michael: WDYT?
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >     -Matthias
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>>>>     >
>>>>     >     > Hi Jay,
>>>>     >
>>>>     >     >           Thanks for the advise, I would like to list
>>>> down
>>> the use cases as
>>>>
>>>>     >
>>>>     >     > per your suggestion. But it seems I don't have write
>>> permission to the
>>>>
>>>>     >
>>>>     >     > Apache Kafka Confluent Space. Whom shall I request
>>>> for it?
>>>>     >
>>>>     >     >
>>>>     >
>>>>     >     > Regarding your last question. We are using a patch in
>>>> our
>>> production system
>>>>
>>>>     >
>>>>     >     > which does exactly this.
>>>>     >
>>>>     >     > We window by the event time, but trigger punctuate in
>>> <punctuate interval>
>>>>
>>>>     >
>>>>     >     > duration of system time, in the absence of an event
>>>> crossing
>>> the punctuate
>>>>
>>>>     >
>>>>     >     > event time.
>>>>     >
>>>>     >     >
>>>>     >
>>>>     >     > We are using Kafka Stream for our Audit Trail, where
>>>> we need
>>> to output the
>>>>
>>>>     >
>>>>     >     > event counts on each topic on each cluster aggregated
>>>> over a
>>> 1 minute
>>>>
>>>>     >
>>>>     >     > window. We have to use event time to be able to cross
>>>> check
>>> the counts. But
>>>>
>>>>     >
>>>>     >     > we need to trigger punctuate [aggregate event pushes]
>>>> by
>>> system time in the
>>>>
>>>>     >
>>>>     >     > absence of events. Otherwise the event counts for
>>>> unexpired
>>> windows would
>>>>
>>>>     >
>>>>     >     > be 0 which is bad.
>>>>     >
>>>>     >     >
>>>>     >
>>>>     >     > "Maybe a hybrid solution works: I window by event
>>>> time but
>>> trigger results
>>>>
>>>>     >
>>>>     >     > by system time for windows that have updated? Not
>>>> really sure
>>> the details
>>>>
>>>>     >
>>>>     >     > of making that work. Does that work? Are there
>>>> concrete
>>> examples where you
>>>>
>>>>     >
>>>>     >     > actually want the current behavior?"
>>>>     >
>>>>     >     >
>>>>     >
>>>>     >     > --
>>>>     >
>>>>     >     > With Regards,
>>>>     >
>>>>     >     >
>>>>     >
>>>>     >     > Arun Mathew
>>>>     >
>>>>     >     > Yahoo! JAPAN Corporation
>>>>     >
>>>>     >     >
>>>>     >
>>>>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
>>> skyahead@gmail.com>
>>> <sk...@gmail.com> wrote:
>>>>
>>>>     >
>>>>     >     >
>>>>     >
>>>>     >     >> Hi Jay,
>>>>     >
>>>>     >     >>
>>>>     >
>>>>     >     >> The hybrid solution is exactly what I expect and
>>>> need for
>>> our use cases
>>>>
>>>>     >
>>>>     >     >> when dealing with telecom data.
>>>>     >
>>>>     >     >>
>>>>     >
>>>>     >     >> Thanks
>>>>     >
>>>>     >     >> Tianji
>>>>     >
>>>>     >     >>
>>>>     >
>>>>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
>>> jay@confluent.io>
>>> <ja...@confluent.io> wrote:
>>>>
>>>>     >
>>>>     >     >>
>>>>     >
>>>>     >     >>> Hey guys,
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>> One thing I've always found super important for
>>>> this kind
>>> of design work
>>>>
>>>>     >
>>>>     >     >> is
>>>>     >
>>>>     >     >>> to do a really good job of cataloging the landscape
>>>> of use
>>> cases and how
>>>>
>>>>     >
>>>>     >     >>> prevalent each one is. By that I mean not just
>>>> listing lots
>>> of uses, but
>>>>
>>>>     >
>>>>     >     >>> also grouping them into categories that
>>>> functionally need
>>> the same thing.
>>>>
>>>>     >
>>>>     >     >>> In the absence of this it is very hard to reason
>>>> about
>>> design proposals.
>>>>
>>>>     >
>>>>     >     >>> From the proposals so far I think we have a lot of
>>> discussion around
>>>>
>>>>     >
>>>>     >     >>> possible apis, but less around what the user needs
>>>> for
>>> different use
>>>>
>>>>     >
>>>>     >     >> cases
>>>>     >
>>>>     >     >>> and how they would implement that using the api.
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>> Here is an example:
>>>>     >
>>>>     >     >>> You aggregate click and impression data for a
>>>> reddit like
>>> site. Every ten
>>>>
>>>>     >
>>>>     >     >>> minutes you want to output a ranked list of the top
>>>> 10
>>> articles ranked by
>>>>
>>>>     >
>>>>     >     >>> clicks/impressions for each geographical area. I
>>>> want to be
>>> able run this
>>>>
>>>>     >
>>>>     >     >>> in steady state as well as rerun to regenerate
>>>> results (or
>>> catch up if it
>>>>
>>>>     >
>>>>     >     >>> crashes).
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>> There are a couple of tricky things that seem to
>>>> make this
>>> hard with
>>>>
>>>>     >
>>>>     >     >> either
>>>>     >
>>>>     >     >>> of the options proposed:
>>>>     >
>>>>     >     >>> 1. If I emit this data using event time I have the
>>>> problem
>>> described
>>>>
>>>>     >
>>>>     >     >> where
>>>>     >
>>>>     >     >>> a geographical region with no new clicks or
>>>> impressions
>>> will fail to
>>>>
>>>>     >
>>>>     >     >> output
>>>>     >
>>>>     >     >>> results.
>>>>     >
>>>>     >     >>> 2. If I emit this data using system time I have the
>>>> problem
>>> that when
>>>>
>>>>     >
>>>>     >     >>> reprocessing data my window may not be ten minutes
>>>> but 10
>>> hours if my
>>>>
>>>>     >
>>>>     >     >>> processing is very fast so it dramatically changes
>>>> the
>>> output.
>>>>
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>> Maybe a hybrid solution works: I window by event
>>>> time but
>>> trigger results
>>>>
>>>>     >
>>>>     >     >>> by system time for windows that have updated? Not
>>>> really
>>> sure the details
>>>>
>>>>     >
>>>>     >     >>> of making that work. Does that work? Are there
>>>> concrete
>>> examples where
>>>>
>>>>     >
>>>>     >     >> you
>>>>     >
>>>>     >     >>> actually want the current behavior?
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>> -Jay
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
>>> arunmathew88@gmail.com> <ar...@gmail.com>
>>>>
>>>>     >
>>>>     >     >>> wrote:
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>>> Hi All,
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> Thanks for the KIP. We were also in need of a
>>>> mechanism to
>>> trigger
>>>>
>>>>     >
>>>>     >     >>>> punctuate in the absence of events.
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> As I described in [
>>>>     >
>>>>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>>     >
>>>>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>>     >
>>>>     >     >>>> plugin.system.issuetabpanels:comment-
>>>> tabpanel#comment-
>>> 15926036
>>>>
>>>>     >
>>>>     >     >>>> ],
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>>    - Our approached involved using the event time
>>>> by
>>> default.
>>>>
>>>>     >
>>>>     >     >>>>    - The method to check if there is any punctuate
>>>> ready
>>> in the
>>>>
>>>>     >
>>>>     >     >>>>    PunctuationQueue is triggered via the any event
>>> received by the
>>>>
>>>>     >
>>>>     >     >> stream
>>>>     >
>>>>     >     >>>>    tread, or at the polling intervals in the
>>>> absence of
>>> any events.
>>>>
>>>>     >
>>>>     >     >>>>    - When we create Punctuate objects (which
>>>> contains the
>>> next event
>>>>
>>>>     >
>>>>     >     >> time
>>>>     >
>>>>     >     >>>>    for punctuation and interval), we also record
>>>> the
>>> creation time
>>>>
>>>>     >
>>>>     >     >>> (system
>>>>     >
>>>>     >     >>>>    time).
>>>>     >
>>>>     >     >>>>    - While checking for maturity of Punctuate
>>>> Schedule by
>>>>     >
>>>>     >     >> mayBePunctuate
>>>>     >
>>>>     >     >>>>    method, we also check if the system clock has
>>>> elapsed
>>> the punctuate
>>>>
>>>>     >
>>>>     >     >>>>    interval since the schedule creation time.
>>>>     >
>>>>     >     >>>>    - In the absence of any event, or in the
>>>> absence of any
>>> event for
>>>>
>>>>     >
>>>>     >     >> one
>>>>     >
>>>>     >     >>>>    topic in the partition group assigned to the
>>>> stream
>>> task, the system
>>>>
>>>>     >
>>>>     >     >>>> time
>>>>     >
>>>>     >     >>>>    will elapse the interval and we trigger a
>>>> punctuate
>>> using the
>>>>
>>>>     >
>>>>     >     >> expected
>>>>     >
>>>>     >     >>>>    punctuation event time.
>>>>     >
>>>>     >     >>>>    - we then create the next punctuation schedule
>>>> as
>>> punctuation event
>>>>
>>>>     >
>>>>     >     >>> time
>>>>     >
>>>>     >     >>>>    + punctuation interval, [again recording the
>>>> system
>>> time of creation
>>>>
>>>>     >
>>>>     >     >>> of
>>>>     >
>>>>     >     >>>> the
>>>>     >
>>>>     >     >>>>    schedule].
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> We call this a Hybrid Punctuate. Of course, this
>>>> approach
>>> has pros and
>>>>
>>>>     >
>>>>     >     >>>> cons.
>>>>     >
>>>>     >     >>>> Pros
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>>    - Punctuates will happen in <punctuate
>>>> interval> time
>>> duration at
>>>>
>>>>     >
>>>>     >     >> max
>>>>     >
>>>>     >     >>> in
>>>>     >
>>>>     >     >>>>    terms of system time.
>>>>     >
>>>>     >     >>>>    - The semantics as a whole continues to revolve
>>>> around
>>> event time.
>>>>
>>>>     >
>>>>     >     >>>>    - We can use the old data [old timestamps] to
>>>> rerun any
>>> experiments
>>>>
>>>>     >
>>>>     >     >> or
>>>>     >
>>>>     >     >>>>    tests.
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> Cons
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>>    - In case the  <punctuate interval> is not a
>>>> time
>>> duration [say
>>>>
>>>>     >
>>>>     >     >>> logical
>>>>     >
>>>>     >     >>>>    time/event count], then the approach might not
>>>> be
>>> meaningful.
>>>>
>>>>     >
>>>>     >     >>>>    - In case there is a case where we have to wait
>>>> for an
>>> actual event
>>>>
>>>>     >
>>>>     >     >>> from
>>>>     >
>>>>     >     >>>>    a low event rate partition in the partition
>>>> group, this
>>> approach
>>>>
>>>>     >
>>>>     >     >> will
>>>>     >
>>>>     >     >>>> jump
>>>>     >
>>>>     >     >>>>    the gun.
>>>>     >
>>>>     >     >>>>    - in case the event processing cannot catch up
>>>> with the
>>> event rate
>>>>
>>>>     >
>>>>     >     >> and
>>>>     >
>>>>     >     >>>>    the expected timestamp events gets queued for
>>>> long
>>> time, this
>>>>
>>>>     >
>>>>     >     >> approach
>>>>     >
>>>>     >     >>>>    might jump the gun.
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> I believe the above approach and discussion goes
>>>> close to
>>> the approach
>>>>
>>>>     >
>>>>     >     >> A.
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> -----------
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> I like the idea of having an even count based
>>>> punctuate.
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> -----------
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> I agree with the discussion around approach C,
>>>> that we
>>> should provide
>>>>
>>>>     >
>>>>     >     >> the
>>>>     >
>>>>     >     >>>> user with the option to choose system time or
>>>> event time
>>> based
>>>>
>>>>     >
>>>>     >     >>> punctuates.
>>>>     >
>>>>     >     >>>> But I believe that the user predominantly wants to
>>>> use
>>> event time while
>>>>
>>>>     >
>>>>     >     >>> not
>>>>     >
>>>>     >     >>>> missing out on regular punctuates due to event
>>>> delays or
>>> event
>>>>
>>>>     >
>>>>     >     >> absences.
>>>>     >
>>>>     >     >>>> Hence a complex punctuate option as Matthias
>>>> mentioned
>>> (quoted below)
>>>>
>>>>     >
>>>>     >     >>> would
>>>>     >
>>>>     >     >>>> be most apt.
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> "- We might want to add "complex" schedules later
>>>> on
>>> (like, punctuate
>>>>
>>>>     >
>>>>     >     >> on
>>>>     >
>>>>     >     >>>> every 10 seconds event-time or 60 seconds system-
>>>> time
>>> whatever comes
>>>>
>>>>     >
>>>>     >     >>>> first)."
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> -----------
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> I think I read somewhere that Kafka Streams
>>>> started with
>>> System Time as
>>>>
>>>>     >
>>>>     >     >>> the
>>>>     >
>>>>     >     >>>> punctuation standard, but was later changed to
>>>> Event Time.
>>> I guess
>>>>
>>>>     >
>>>>     >     >> there
>>>>     >
>>>>     >     >>>> would be some good reason behind it. As Kafka
>>>> Streams want
>>> to evolve
>>>>
>>>>     >
>>>>     >     >> more
>>>>     >
>>>>     >     >>>> on the Stream Processing front, I believe the
>>>> emphasis on
>>> event time
>>>>
>>>>     >
>>>>     >     >>> would
>>>>     >
>>>>     >     >>>> remain quite strong.
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> With Regards,
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> Arun Mathew
>>>>     >
>>>>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
>>> tobecker@tivo.com> <to...@tivo.com>
>>>>
>>>>     >
>>>>     >     >> wrote:
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>>> Yeah I like PuncutationType much better; I just
>>>> threw
>>> Time out there
>>>>
>>>>     >
>>>>     >     >>>>> more as a strawman than an actual suggestion ;) I
>>>> still
>>> think it's
>>>>
>>>>     >
>>>>     >     >>>>> worth considering what this buys us over an
>>>> additional
>>> callback. I
>>>>
>>>>     >
>>>>     >     >>>>> foresee a number of punctuate implementations
>>>> following
>>> this pattern:
>>>>
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>> public void punctuate(PunctuationType type) {
>>>>     >
>>>>     >     >>>>>     switch (type) {
>>>>     >
>>>>     >     >>>>>         case EVENT_TIME:
>>>>     >
>>>>     >     >>>>>             methodA();
>>>>     >
>>>>     >     >>>>>             break;
>>>>     >
>>>>     >     >>>>>         case SYSTEM_TIME:
>>>>     >
>>>>     >     >>>>>             methodB();
>>>>     >
>>>>     >     >>>>>             break;
>>>>     >
>>>>     >     >>>>>     }
>>>>     >
>>>>     >     >>>>> }
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>> I guess one advantage of this approach is we
>>>> could add
>>> additional
>>>>
>>>>     >
>>>>     >     >>>>> punctuation types later in a backwards compatible
>>>> way
>>> (like event
>>>>
>>>>     >
>>>>     >     >> count
>>>>     >
>>>>     >     >>>>> as you mentioned).
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>> -Tommy
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
>>>> Sax wrote:
>>>>     >
>>>>     >     >>>>>> That sounds promising.
>>>>     >
>>>>     >     >>>>>>
>>>>     >
>>>>     >     >>>>>> I am just wondering if `Time` is the best name.
>>>> Maybe we
>>> want to
>>>>
>>>>     >
>>>>     >     >> add
>>>>     >
>>>>     >     >>>>>> other non-time based punctuations at some point
>>>> later. I
>>> would
>>>>
>>>>     >
>>>>     >     >>>>>> suggest
>>>>     >
>>>>     >     >>>>>>
>>>>     >
>>>>     >     >>>>>> enum PunctuationType {
>>>>     >
>>>>     >     >>>>>>   EVENT_TIME,
>>>>     >
>>>>     >     >>>>>>   SYSTEM_TIME,
>>>>     >
>>>>     >     >>>>>> }
>>>>     >
>>>>     >     >>>>>>
>>>>     >
>>>>     >     >>>>>> or similar. Just to keep the door open -- it's
>>>> easier to
>>> add new
>>>>
>>>>     >
>>>>     >     >>>>>> stuff
>>>>     >
>>>>     >     >>>>>> if the name is more generic.
>>>>     >
>>>>     >     >>>>>>
>>>>     >
>>>>     >     >>>>>>
>>>>     >
>>>>     >     >>>>>> -Matthias
>>>>     >
>>>>     >     >>>>>>
>>>>     >
>>>>     >     >>>>>>
>>>>     >
>>>>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>> I agree that the framework providing and
>>>> managing the
>>> notion of
>>>>
>>>>     >
>>>>     >     >>>>>>> stream
>>>>     >
>>>>     >     >>>>>>> time is valuable and not something we would
>>>> want to
>>> delegate to
>>>>
>>>>     >
>>>>     >     >> the
>>>>     >
>>>>     >     >>>>>>> tasks. I'm not entirely convinced that a
>>>> separate
>>> callback
>>>>
>>>>     >
>>>>     >     >> (option
>>>>     >
>>>>     >     >>>>>>> C)
>>>>     >
>>>>     >     >>>>>>> is that messy (it could just be a default
>>>> method with
>>> an empty
>>>>
>>>>     >
>>>>     >     >>>>>>> implementation), but if we wanted a single API
>>>> to
>>> handle both
>>>>
>>>>     >
>>>>     >     >>>>>>> cases,
>>>>     >
>>>>     >     >>>>>>> how about something like the following?
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>> enum Time {
>>>>     >
>>>>     >     >>>>>>>    STREAM,
>>>>     >
>>>>     >     >>>>>>>    CLOCK
>>>>     >
>>>>     >     >>>>>>> }
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>> Then on ProcessorContext:
>>>>     >
>>>>     >     >>>>>>> context.schedule(Time time, long interval)  //
>>>> We could
>>> allow
>>>>
>>>>     >
>>>>     >     >> this
>>>>     >
>>>>     >     >>>>>>> to
>>>>     >
>>>>     >     >>>>>>> be called once for each value of time to mix
>>> approaches.
>>>>
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>> Then the Processor API becomes:
>>>>     >
>>>>     >     >>>>>>> punctuate(Time time) // time here denotes which
>>> schedule resulted
>>>>
>>>>     >
>>>>     >     >>>>>>> in
>>>>     >
>>>>     >     >>>>>>> this call.
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>> Thoughts?
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
>>>> Sax
>>> wrote:
>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> I was thinking about the four options you
>>>> proposed in
>>> more
>>>>
>>>>     >
>>>>     >     >>>>>>>> details
>>>>     >
>>>>     >     >>>>>>>> and
>>>>     >
>>>>     >     >>>>>>>> this are my thoughts:
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> (A) You argue, that users can still
>>>> "punctuate" on
>>> event-time
>>>>
>>>>     >
>>>>     >     >> via
>>>>     >
>>>>     >     >>>>>>>> process(), but I am not sure if this is
>>>> possible.
>>> Note, that
>>>>
>>>>     >
>>>>     >     >>>>>>>> users
>>>>     >
>>>>     >     >>>>>>>> only
>>>>     >
>>>>     >     >>>>>>>> get record timestamps via context.timestamp().
>>>> Thus,
>>> users
>>>>
>>>>     >
>>>>     >     >> would
>>>>     >
>>>>     >     >>>>>>>> need
>>>>     >
>>>>     >     >>>>>>>> to
>>>>     >
>>>>     >     >>>>>>>> track the time progress per partition (based
>>>> on the
>>> partitions
>>>>
>>>>     >
>>>>     >     >>>>>>>> they
>>>>     >
>>>>     >     >>>>>>>> obverse via context.partition(). (This alone
>>>> puts a
>>> huge burden
>>>>
>>>>     >
>>>>     >     >>>>>>>> on
>>>>     >
>>>>     >     >>>>>>>> the
>>>>     >
>>>>     >     >>>>>>>> user by itself.) However, users are not
>>>> notified at
>>> startup
>>>>
>>>>     >
>>>>     >     >> what
>>>>     >
>>>>     >     >>>>>>>> partitions are assigned, and user are not
>>>> notified
>>> when
>>>>
>>>>     >
>>>>     >     >>>>>>>> partitions
>>>>     >
>>>>     >     >>>>>>>> get
>>>>     >
>>>>     >     >>>>>>>> revoked. Because this information is not
>>>> available,
>>> it's not
>>>>
>>>>     >
>>>>     >     >>>>>>>> possible
>>>>     >
>>>>     >     >>>>>>>> to
>>>>     >
>>>>     >     >>>>>>>> "manually advance" stream-time, and thus
>>>> event-time
>>> punctuation
>>>>
>>>>     >
>>>>     >     >>>>>>>> within
>>>>     >
>>>>     >     >>>>>>>> process() seems not to be possible -- or do
>>>> you see a
>>> way to
>>>>
>>>>     >
>>>>     >     >> get
>>>>     >
>>>>     >     >>>>>>>> it
>>>>     >
>>>>     >     >>>>>>>> done? And even if, it might still be too
>>>> clumsy to
>>> use.
>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> (B) This does not allow to mix both
>>>> approaches, thus
>>> limiting
>>>>
>>>>     >
>>>>     >     >>>>>>>> what
>>>>     >
>>>>     >     >>>>>>>> users
>>>>     >
>>>>     >     >>>>>>>> can do.
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> (C) This should give all flexibility we need.
>>>> However,
>>> just
>>>>
>>>>     >
>>>>     >     >>>>>>>> adding
>>>>     >
>>>>     >     >>>>>>>> one
>>>>     >
>>>>     >     >>>>>>>> more method seems to be a solution that is too
>>>> simple
>>> (cf my
>>>>
>>>>     >
>>>>     >     >>>>>>>> comments
>>>>     >
>>>>     >     >>>>>>>> below).
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> (D) This might be hard to use. Also, I am not
>>>> sure how
>>> a user
>>>>
>>>>     >
>>>>     >     >>>>>>>> could
>>>>     >
>>>>     >     >>>>>>>> enable system-time and event-time punctuation
>>>> in
>>> parallel.
>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> Overall options (C) seems to be the most
>>>> promising
>>> approach to
>>>>
>>>>     >
>>>>     >     >>>>>>>> me.
>>>>     >
>>>>     >     >>>>>>>> Because I also favor a clean API, we might
>>>> keep
>>> current
>>>>
>>>>     >
>>>>     >     >>>>>>>> punctuate()
>>>>     >
>>>>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it
>>>> at some
>>> later
>>>>
>>>>     >
>>>>     >     >>>>>>>> point
>>>>     >
>>>>     >     >>>>>>>> when
>>>>     >
>>>>     >     >>>>>>>> people use the "new punctuate API".
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> Couple of follow up questions:
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> - I am wondering, if we should have two
>>>> callback
>>> methods or
>>>>
>>>>     >
>>>>     >     >> just
>>>>     >
>>>>     >     >>>>>>>> one
>>>>     >
>>>>     >     >>>>>>>> (ie, a unified for system and event time
>>>> punctuation
>>> or one for
>>>>
>>>>     >
>>>>     >     >>>>>>>> each?).
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> - If we have one, how can the user figure out,
>>>> which
>>> condition
>>>>
>>>>     >
>>>>     >     >>>>>>>> did
>>>>     >
>>>>     >     >>>>>>>> trigger?
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> - How would the API look like, for registering
>>> different
>>>>
>>>>     >
>>>>     >     >>>>>>>> punctuate
>>>>     >
>>>>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> - We might want to add "complex" schedules
>>>> later on
>>> (like,
>>>>
>>>>     >
>>>>     >     >>>>>>>> punctuate
>>>>     >
>>>>     >     >>>>>>>> on
>>>>     >
>>>>     >     >>>>>>>> every 10 seconds event-time or 60 seconds
>>>> system-time
>>> whatever
>>>>
>>>>     >
>>>>     >     >>>>>>>> comes
>>>>     >
>>>>     >     >>>>>>>> first). I don't say we should add this right
>>>> away, but
>>> we might
>>>>
>>>>     >
>>>>     >     >>>>>>>> want
>>>>     >
>>>>     >     >>>>>>>> to
>>>>     >
>>>>     >     >>>>>>>> define the API in a way, that it allows
>>>> extensions
>>> like this
>>>>
>>>>     >
>>>>     >     >>>>>>>> later
>>>>     >
>>>>     >     >>>>>>>> on,
>>>>     >
>>>>     >     >>>>>>>> without redesigning the API (ie, the API
>>>> should be
>>> designed
>>>>
>>>>     >
>>>>     >     >>>>>>>> extensible)
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> - Did you ever consider count-based
>>>> punctuation?
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> I understand, that you would like to solve a
>>>> simple
>>> problem,
>>>>
>>>>     >
>>>>     >     >> but
>>>>     >
>>>>     >     >>>>>>>> we
>>>>     >
>>>>     >     >>>>>>>> learned from the past, that just "adding some
>>>> API"
>>> quickly
>>>>
>>>>     >
>>>>     >     >> leads
>>>>     >
>>>>     >     >>>>>>>> to a
>>>>     >
>>>>     >     >>>>>>>> not very well defined API that needs time
>>>> consuming
>>> clean up
>>>>
>>>>     >
>>>>     >     >>>>>>>> later on
>>>>     >
>>>>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
>>>> holistic
>>>>     >
>>>>     >     >>>>>>>> punctuation
>>>>     >
>>>>     >     >>>>>>>> KIP
>>>>     >
>>>>     >     >>>>>>>> with this from the beginning on to avoid later
>>>> painful
>>>>     >
>>>>     >     >> redesign.
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> -Matthias
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>
>>>>     >
>>>>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>> Thanks Thomas,
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>> I'm also wary of changing the existing
>>>> semantics of
>>>>     >
>>>>     >     >> punctuate,
>>>>     >
>>>>     >     >>>>>>>>> for
>>>>     >
>>>>     >     >>>>>>>>> backward compatibility reasons, although I
>>>> like the
>>>>     >
>>>>     >     >> conceptual
>>>>     >
>>>>     >     >>>>>>>>> simplicity of that option.
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>> Adding a new method to me feels safer but, in
>>>> a way,
>>> uglier.
>>>>
>>>>     >
>>>>     >     >> I
>>>>     >
>>>>     >     >>>>>>>>> added
>>>>     >
>>>>     >     >>>>>>>>> this to the KIP now as option (C).
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>> The TimestampExtractor mechanism is actually
>>>> more
>>> flexible,
>>>>
>>>>     >
>>>>     >     >> as
>>>>     >
>>>>     >     >>>>>>>>> it
>>>>     >
>>>>     >     >>>>>>>>> allows
>>>>     >
>>>>     >     >>>>>>>>> you to return any value, you're not limited
>>>> to event
>>> time or
>>>>
>>>>     >
>>>>     >     >>>>>>>>> system
>>>>     >
>>>>     >     >>>>>>>>> time
>>>>     >
>>>>     >     >>>>>>>>> (although I don't see an actual use case
>>>> where you
>>> might need
>>>>
>>>>     >
>>>>     >     >>>>>>>>> anything
>>>>     >
>>>>     >     >>>>>>>>> else then those two). Hence I also proposed
>>>> the
>>> option to
>>>>
>>>>     >
>>>>     >     >> allow
>>>>     >
>>>>     >     >>>>>>>>> users
>>>>     >
>>>>     >     >>>>>>>>> to, effectively, decide what "stream time" is
>>>> for
>>> them given
>>>>
>>>>     >
>>>>     >     >>>>>>>>> the
>>>>     >
>>>>     >     >>>>>>>>> presence or absence of messages, much like
>>>> they can
>>> decide
>>>>
>>>>     >
>>>>     >     >> what
>>>>     >
>>>>     >     >>>>>>>>> msg
>>>>     >
>>>>     >     >>>>>>>>> time
>>>>     >
>>>>     >     >>>>>>>>> means for them using the TimestampExtractor.
>>>> What do
>>> you
>>>>
>>>>     >
>>>>     >     >> think
>>>>     >
>>>>     >     >>>>>>>>> about
>>>>     >
>>>>     >     >>>>>>>>> that? This is probably most flexible but also
>>>> most
>>>>     >
>>>>     >     >> complicated.
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>> All comments appreciated.
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>> Cheers,
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>> Michal
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>     >
>>>>     >     >>>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>> Although I fully agree we need a way to
>>>> trigger
>>> periodic
>>>>
>>>>     >
>>>>     >     >>>>>>>>>> processing
>>>>     >
>>>>     >     >>>>>>>>>> that is independent from whether and when
>>>> messages
>>> arrive,
>>>>
>>>>     >
>>>>     >     >>>>>>>>>> I'm
>>>>     >
>>>>     >     >>>>>>>>>> not sure
>>>>     >
>>>>     >     >>>>>>>>>> I like the idea of changing the existing
>>>> semantics
>>> across
>>>>
>>>>     >
>>>>     >     >> the
>>>>     >
>>>>     >     >>>>>>>>>> board.
>>>>     >
>>>>     >     >>>>>>>>>> What if we added an additional callback to
>>>> Processor
>>> that
>>>>
>>>>     >
>>>>     >     >> can
>>>>     >
>>>>     >     >>>>>>>>>> be
>>>>     >
>>>>     >     >>>>>>>>>> scheduled similarly to punctuate() but was
>>>> always
>>> called at
>>>>
>>>>     >
>>>>     >     >>>>>>>>>> fixed, wall
>>>>     >
>>>>     >     >>>>>>>>>> clock based intervals? This way you wouldn't
>>>> have to
>>> give
>>>>
>>>>     >
>>>>     >     >> up
>>>>     >
>>>>     >     >>>>>>>>>> the
>>>>     >
>>>>     >     >>>>>>>>>> notion
>>>>     >
>>>>     >     >>>>>>>>>> of stream time to be able to do periodic
>>>> processing.
>>>>     >
>>>>     >     >>>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
>>>> Borowiecki
>>> wrote:
>>>>
>>>>     >
>>>>     >     >>>>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>>> Hi all,
>>>>     >
>>>>     >     >>>>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
>>> punctuate
>>>>
>>>>     >
>>>>     >     >>>>>>>>>>> semantics
>>>>     >
>>>>     >     >>>>>>>>>>> <https://cwiki.apache.org/
>>> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
>>> confluence/display/KAFKA/KIP->
>>>>
>>>>     >
>>>>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI
>>>> P->>>
>>> 138%
>>>>
>>>>     >
>>>>     >     >>>>>>>>>>> 3A+C
>>>>     >
>>>>     >     >>>>>>>>>>> hange+
>>>>     >
>>>>     >     >>>>>>>>>>> punctuate+semantics>
>>>>     >
>>>>     >     >>>>>>>>>>> .
>>>>     >
>>>>     >     >>>>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>>> Appreciating there can be different views
>>>> on
>>> system-time
>>>>
>>>>     >
>>>>     >     >> vs
>>>>     >
>>>>     >     >>>>>>>>>>> event-
>>>>     >
>>>>     >     >>>>>>>>>>> time
>>>>     >
>>>>     >     >>>>>>>>>>> semantics for punctuation depending on use-
>>>> case and
>>> the
>>>>
>>>>     >
>>>>     >     >>>>>>>>>>> importance of
>>>>     >
>>>>     >     >>>>>>>>>>> backwards compatibility of any such change,
>>>> I've
>>> left it
>>>>
>>>>     >
>>>>     >     >>>>>>>>>>> quite
>>>>     >
>>>>     >     >>>>>>>>>>> open
>>>>     >
>>>>     >     >>>>>>>>>>> and
>>>>     >
>>>>     >     >>>>>>>>>>> hope to fill in more info as the discussion
>>> progresses.
>>>>
>>>>     >
>>>>     >     >>>>>>>>>>>
>>>>     >
>>>>     >     >>>>>>>>>>> Thanks,
>>>>     >
>>>>     >     >>>>>>>>>>> Michal
>>>>     >
>>>>     >     >>>>>>> --
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>>     Tommy Becker
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>>     Senior Software Engineer
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>>     tivo.com
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>> ________________________________
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>>>> This email and any attachments may contain
>>>> confidential
>>> and
>>>>
>>>>     >
>>>>     >     >>>>>>> privileged material for the sole use of the
>>>> intended
>>> recipient.
>>>>
>>>>     >
>>>>     >     >> Any
>>>>     >
>>>>     >     >>>>>>> review, copying, or distribution of this email
>>>> (or any
>>>>     >
>>>>     >     >> attachments)
>>>>     >
>>>>     >     >>>>>>> by others is prohibited. If you are not the
>>>> intended
>>> recipient,
>>>>
>>>>     >
>>>>     >     >>>>>>> please contact the sender immediately and
>>>> permanently
>>> delete this
>>>>
>>>>     >
>>>>     >     >>>>>>> email and any attachments. No employee or agent
>>>> of TiVo
>>> Inc. is
>>>>
>>>>     >
>>>>     >     >>>>>>> authorized to conclude any binding agreement on
>>>> behalf
>>> of TiVo
>>>>
>>>>     >
>>>>     >     >> Inc.
>>>>     >
>>>>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may
>>>> only be
>>> made by a
>>>>
>>>>     >
>>>>     >     >>>>>>> signed written agreement.
>>>>     >
>>>>     >     >>>>>>>
>>>>     >
>>>>     >     >>>>> --
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>>     Tommy Becker
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>>     Senior Software Engineer
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>>     tivo.com
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>> ________________________________
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>> This email and any attachments may contain
>>>> confidential
>>> and
>>>>
>>>>     >
>>>>     >     >> privileged
>>>>     >
>>>>     >     >>>>> material for the sole use of the intended
>>>> recipient. Any
>>> review,
>>>>
>>>>     >
>>>>     >     >>> copying,
>>>>     >
>>>>     >     >>>>> or distribution of this email (or any
>>>> attachments) by
>>> others is
>>>>
>>>>     >
>>>>     >     >>>> prohibited.
>>>>     >
>>>>     >     >>>>> If you are not the intended recipient, please
>>>> contact the
>>> sender
>>>>
>>>>     >
>>>>     >     >>>>> immediately and permanently delete this email and
>>>> any
>>> attachments. No
>>>>
>>>>     >
>>>>     >     >>>>> employee or agent of TiVo Inc. is authorized to
>>>> conclude
>>> any binding
>>>>
>>>>     >
>>>>     >     >>>>> agreement on behalf of TiVo Inc. by email.
>>>> Binding
>>> agreements with
>>>>
>>>>     >
>>>>     >     >> TiVo
>>>>     >
>>>>     >     >>>>> Inc. may only be made by a signed written
>>>> agreement.
>>>>     >
>>>>     >     >>>>>
>>>>     >
>>>>     >     >>>>
>>>>     >
>>>>     >     >>>
>>>>     >
>>>>     >     >>
>>>>     >
>>>>     >     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     >
>>>>     > --
>>>>     >
>>>>     > <http://www.openbet.com/>
>>>>     >
>>>>     > *Michal Borowiecki*
>>>>     >
>>>>     > *Senior Software Engineer L4*
>>>>     >
>>>>     > *T: *
>>>>     >
>>>>     > +44 208 742 1600 <+44%2020%208742%201600>
>>>>     >
>>>>     > +44 203 249 8448 <+44%2020%203249%208448>
>>>>     >
>>>>     >
>>>>     >
>>>>     > *E: *
>>>>     >
>>>>     > michal.borowiecki@openbet.com
>>>>     >
>>>>     > *W: *
>>>>     >
>>>>     > www.openbet.com
>>>>     >
>>>>     > *OpenBet Ltd*
>>>>     >
>>>>     > Chiswick Park Building 9
>>>>     >
>>>>     > 566 Chiswick High Rd
>>>>     >
>>>>     > London
>>>>     >
>>>>     > W4 5XT
>>>>     >
>>>>     > UK
>>>>     >
>>>>     > <https://www.openbet.com/email_promo>
>>>>     >
>>>>     >
>>>>     >
>>>>     > This message is confidential and intended only for the
>>>> addressee.
>>> If you
>>>>
>>>>     > have received this message in error, please immediately
>>>> notify the
>>>>     > postmaster@openbet.com and delete it from your system as
>>>> well as
>>> any
>>>>
>>>>     > copies. The content of e-mails as well as traffic data may
>>>> be
>>> monitored by
>>>>
>>>>     > OpenBet for employment and security purposes. To protect
>>>> the
>>> environment
>>>>
>>>>     > please do not print this e-mail unless necessary. OpenBet
>>>> Ltd.
>>> Registered
>>>>
>>>>     > Office: Chiswick Park Building 9, 566 Chiswick High Road,
>>>> London,
>>> W4 5XT,
>>>>
>>>>     > United Kingdom. A company registered in England and Wales.
>>> Registered no.
>>>>
>>>>     > 3134634. VAT no. GB927523612
>>>>     >
>>>>     >
>>>>     >
>>>>
>>>>
> --
> 
> 
>     Tommy Becker
> 
>     Senior Software Engineer
> 
>     O +1 919.460.4747
> 
>     tivo.com
> 
> 
> ________________________________
> 
> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Thomas Becker <to...@Tivo.com>.
Here's an example that we currently have.  We have a streams processor
that does a transform from one topic into another. One of the fields in
the source topic record is an expiration time, and one of the functions
of the processor is to ensure that expired records get deleted promptly
after that time passes (typically days or weeks after the message was
originally produced). To do that, the processor keeps a state store of
keys and expiration times, iterates that store in punctuate(), and
emits delete (null) records for expired items. This needs to happen at
some minimum interval regardless of the incoming message rate of the
source topic.

In this scenario, the expiration of records is the primary function of
punctuate, and therefore the key requirement is that the wall-clock
measured time between punctuate calls have some upper-bound. So a pure
wall-clock based schedule would be fine for our needs. But the proposed
"hybrid" system would also be acceptable if that satisfies a broader
range of use-cases.

On Tue, 2017-04-11 at 14:41 +0200, Michael Noll wrote:
> I apologize for the longer email below.  To my defense, it started
> out much
> shorter. :-)  Also, to be super-clear, I am intentionally playing
> devil's
> advocate for a number of arguments brought forth in order to help
> improve
> this KIP -- I am not implying I necessarily disagree with the
> arguments.
>
> That aside, here are some further thoughts.
>
> First, there are (at least?) two categories for actions/behavior you
> invoke
> via punctuate():
>
> 1. For internal housekeeping of your Processor or Transformer (e.g.,
> to
> periodically commit to a custom store, to do metrics/logging).  Here,
> the
> impact of punctuate is typically not observable by other processing
> nodes
> in the topology.
> 2. For controlling the emit frequency of downstream records.  Here,
> the
> punctuate is all about being observable by downstream processing
> nodes.
>
> A few releases back, we introduced record caches (DSL) and state
> store
> caches (Processor API) in KIP-63.  Here, we addressed a concern
> relating to
> (2) where some users needed to control -- here: limit -- the
> downstream
> output rate of Kafka Streams because the downstream systems/apps
> would not
> be able to keep up with the upstream output rate (Kafka scalability >
> their
> scalability).  The argument for KIP-63, which notably did not
> introduce a
> "trigger" API, was that such an interaction with downstream systems
> is an
> operational concern;  it should not impact the processing *logic* of
> your
> application, and thus we didn't want to complicate the Kafka Streams
> API,
> especially not the declarative DSL, with such operational concerns.
>
> This KIP's discussion on `punctuate()` takes us back in time (<--
> sorry, I
> couldn't resist to not make this pun :-P).  As a meta-comment, I am
> observing that our conversation is moving more and more into the
> direction
> of explicit "triggers" because, so far, I have seen only motivations
> for
> use cases in category (2), but none yet for (1)?  For example, some
> comments voiced here are about sth like "IF stream-time didn't
> trigger
> punctuate, THEN trigger punctuate based on processing-time".  Do we
> want
> this, and if so, for which use cases and benefits?  Also, on a
> related
> note, whatever we are discussing here will impact state store caches
> (Processor API) and perhaps also impact record caches (DSL), thus we
> should
> clarify any such impact here.
>
> Switching topics slightly.
>
> Jay wrote:
> >
> > One thing I've always found super important for this kind of design
> > work
> > is to do a really good job of cataloging the landscape of use cases
> > and
> > how prevalent each one is.
> +1 to this, as others have already said.
>
> Here, let me highlight -- just in case -- that when we talked about
> windowing use cases in the recent emails, the Processor API (where
> `punctuate` resides) does not have any notion of windowing at
> all.  If you
> want to do windowing *in the Processor API*, you must do so manually
> in
> combination with window stores.  For this reason I'd suggest to
> discuss use
> cases not just in general, but also in view of how you'd do so in the
> Processor API vs. in the DSL.  Right now, changing/improving
> `punctuate`
> does not impact the DSL at all, unless we add new functionality to
> it.
>
> Jay wrote in his strawman example:
> >
> > You aggregate click and impression data for a reddit like site.
> > Every ten
> > minutes you want to output a ranked list of the top 10 articles
> > ranked by
> > clicks/impressions for each geographical area. I want to be able
> > run this
> > in steady state as well as rerun to regenerate results (or catch up
> > if it
> > crashes).
> This is a good example for more than the obvious reason:  In KIP-63,
> we
> argued that the reason for saying "every ten minutes" above is not
> necessarily about because you want to output data *exactly* after ten
> minutes, but that you want to perform an aggregation based on 10-
> minute
> windows of input data; i.e., the point is about specifying the input
> for
> your aggregation, not or less about when the results of the
> aggregation
> should be send downstream.  To take an extreme example, you could
> disable
> record caches and let your app output a downstream update for every
> incoming input record.  If the last input record was from at minute 7
> of 10
> (for a 10-min window), then what your app would output at minute 10
> would
> be identical to what it had already emitted at minute 7 earlier
> anyways.
> This is particularly true when we take late-arriving data into
> account:  if
> a late record arrived at minute 13, your app would (by default) send
> a new
> update downstream, even though the "original" 10 minutes have already
> passed.
>
> Jay wrote...:
> >
> > There are a couple of tricky things that seem to make this hard
> > with
> either
> >
> > of the options proposed:
> > 1. If I emit this data using event time I have the problem
> > described where
> > a geographical region with no new clicks or impressions will fail
> > to
> output
> >
> > results.
> ...and Arun Mathew wrote:
>
> >
> > We window by the event time, but trigger punctuate in <punctuate
> > interval>
> > duration of system time, in the absence of an event crossing the
> > punctuate
> > event time.
> So, given what I wrote above about the status quo and what you can
> already
> do with it, is the concern that the state store cache doesn't give
> you
> *direct* control over "forcing an output after no later than X
> seconds [of
> processing-time]" but only indirect control through a cache
> size?  (Note
> that I am not dismissing the claims why this might be helpful.)
>
> Arun Mathew wrote:
> >
> > We are using Kafka Stream for our Audit Trail, where we need to
> > output the
> > event counts on each topic on each cluster aggregated over a 1
> > minute
> > window. We have to use event time to be able to cross check the
> > counts.
> But
> >
> > we need to trigger punctuate [aggregate event pushes] by system
> > time in
> the
> >
> > absence of events. Otherwise the event counts for unexpired windows
> > would
> > be 0 which is bad.
> Isn't the latter -- "count would be 0" -- the problem between the
> absence
> of output vs. an output of 0, similar to the use of `Option[T]` in
> Scala
> and the difference between `None` and `Some(0)`?  That is, isn't the
> root
> cause that the downstream system interprets the absence of output in
> a
> particular way ("No output after 1 minute = I consider the output to
> be
> 0.")?  Arguably, you could also adapt the downstream system (if
> possible)
> to correctly handle the difference between absence of output vs.
> output of
> 0.  I am not implying that we shouldn't care about such a use case,
> but
> want to understand the motivation better. :-)
>
> Also, to add some perspective, in some related discussions we talked
> about
> how a Kafka Streams application should not worry or not be coupled
> unnecessarily with such interpretation specifics in a downstream
> system's
> behavior.  After all, tomorrow your app's output might be consumed by
> more
> than just this one downstream system.  Arguably, Kafka Connect rather
> than
> Kafka Streams might be the best tool to link the universes of Kafka
> and
> downstream systems, including helping to reconcile the differences in
> how
> these systems interpret changes, updates, late-arriving data,
> etc.  Kafka
> Connect would allow you to decouple the Kafka Streams app's logical
> processing from the specifics of downstream systems, thanks to
> specific
> sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would
> this
> decoupling with Kafka Connect help here?  (And if the answer is "Yes,
> but
> it's currently awkward to use Connect for this", this might be a
> problem we
> can solve, too.)
>
> Switching topics slightly again.
>
> Thomas wrote:
> >
> > I'm not entirely convinced that a separate callback (option C)
> > is that messy (it could just be a default method with an empty
> > implementation), but if we wanted a single API to handle both
> > cases,
> > how about something like the following?
> >
> > enum Time {
> >    STREAM,
> >    CLOCK
> > }
> Yeah, I am on the fence here, too.  If we use the 1-method approach,
> then
> whatever the user is doing inside this method is a black box to Kafka
> Streams (similar to how we have no idea what the user does inside a
> `foreach` -- if the function passed to `foreach` writes to external
> systems, then Kafka Streams is totally unaware of the fact).  We
> won't
> know, for example, if the stream-time action has a smaller "trigger"
> frequency than the processing-time action.  Or, we won't know whether
> the
> user custom-codes a "not later than" trigger logic ("Do X every 1-
> minute of
> stream-time or 1-minute of processing-time, whichever comes
> first").  That
> said, I am not certain yet whether we would need such knowledge
> because,
> when using the Processor API, most of the work and decisions must be
> done
> by the user anyways.  It would matter though if the concept of
> "triggers"
> were to bubble up into the DSL because in the DSL the management of
> windowing, window stores, etc. must be done automatically by Kafka
> Streams.
>
> [In any case, btw, we have the corner case where the user configured
> the
> stream-time to be processing-time (e.g. via wall-clock timestamp
> extractor), at which point both punctuate variants are based on the
> same
> time semantics / timeline.]
>
> Again, I apologize for the wall of text.  Congratulations if you made
> it
> this far. :-)
>
> More than happy to hear your thoughts!
> Michael
>
> On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com>
> wrote:
>
> >
> > Thanks Matthias.
> > Sure, will correct it right away.
> >
> > On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io>
> > wrote:
> >
> > Thanks for preparing this page!
> >
> > About terminology:
> >
> > You introduce the term "event time" -- but we should call this
> > "stream
> > time" -- "stream time" is whatever TimestampExtractor returns and
> > this
> > could be event time, ingestion time, or processing/wall-clock time.
> >
> > Does this make sense to you?
> >
> >
> >
> > -Matthias
> >
> >
> > On 4/10/17 4:58 AM, Arun Mathew wrote:
> > >
> > > Thanks Ewen.
> > >
> > > @Michal, @all, I have created a child page to start the Use Cases
> > discussion [https://cwiki.apache.org/confluence/display/KAFKA/
> > Punctuate+Use+Cases]. Please go through it and give your comments.
> > >
> > >
> > > @Tianji, Sorry for the delay. I am trying to make the patch
> > > public.
> > >
> > > --
> > > Arun Mathew
> > >
> > > On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io>
> > > wrote:
> > >
> > >     Arun,
> > >
> > >     I've given you permission to edit the wiki. Let me know if
> > > you run
> > into any
> > >
> > >     issues.
> > >
> > >     -Ewen
> > >
> > >     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <amathew@yahoo-co
> > > rp.jp>
> > wrote:
> > >
> > >
> > >     > Thanks Michal. I don’t have the access yet [arunmathew88].
> > > Should I
> > be
> > >
> > >     > sending a separate mail for this?
> > >     >
> > >     > I thought one of the person following this thread would be
> > > able to
> > give me
> > >
> > >     > access.
> > >     >
> > >     >
> > >     >
> > >     > *From: *Michal Borowiecki <mi...@openbet.com>
> > >     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> > >     > *Date: *Friday, April 7, 2017 at 17:16
> > >     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> > >     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate
> > > semantics
> > >     >
> > >     >
> > >     >
> > >     > Hi Arun,
> > >     >
> > >     > I was thinking along the same lines as you, listing the use
> > > cases
> > on the
> > >
> > >     > wiki, but didn't find time to get around doing that yet.
> > >     > Don't mind if you do it if you have access now.
> > >     > I was thinking it would be nice if, once we have the use
> > > cases
> > listed,
> > >
> > >     > people could use likes to up-vote the use cases similar to
> > > what
> > they're
> > >
> > >     > working on.
> > >     >
> > >     > I should have a bit more time to action this in the next
> > > few days,
> > but
> > >
> > >     > happy for you to do it if you can beat me to it ;-)
> > >     >
> > >     > Cheers,
> > >     > Michal
> > >     >
> > >     > On 07/04/17 04:39, Arun Mathew wrote:
> > >     >
> > >     > Sure, Thanks Matthias. My id is [arunmathew88].
> > >     >
> > >     >
> > >     >
> > >     > Of course. I was thinking of a subpage where people can
> > collaborate.
> > >
> > >     >
> > >     >
> > >     >
> > >     > Will do as per Michael’s suggestion.
> > >     >
> > >     >
> > >     >
> > >     > Regards,
> > >     >
> > >     > Arun Mathew
> > >     >
> > >     >
> > >     >
> > >     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io>
> > > <
> > matthias@confluent.io> wrote:
> > >
> > >     >
> > >     >
> > >     >
> > >     >     Please share your Wiki-ID and a committer can give you
> > > write
> > access.
> > >
> > >     >
> > >     >
> > >     >
> > >     >     Btw: as you did not initiate the KIP, you should not
> > > change the
> > KIP
> > >
> > >     >
> > >     >     without the permission of the original author -- in
> > > this case
> > Michael.
> > >
> > >     >
> > >     >
> > >     >
> > >     >     So you might also just share your thought over the
> > > mailing list
> > and
> > >
> > >     >
> > >     >     Michael can update the KIP page. Or, as an alternative,
> > > just
> > create a
> > >
> > >     >
> > >     >     subpage for the KIP page.
> > >     >
> > >     >
> > >     >
> > >     >     @Michael: WDYT?
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >     -Matthias
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
> > >     >
> > >     >     > Hi Jay,
> > >     >
> > >     >     >           Thanks for the advise, I would like to list
> > > down
> > the use cases as
> > >
> > >     >
> > >     >     > per your suggestion. But it seems I don't have write
> > permission to the
> > >
> > >     >
> > >     >     > Apache Kafka Confluent Space. Whom shall I request
> > > for it?
> > >     >
> > >     >     >
> > >     >
> > >     >     > Regarding your last question. We are using a patch in
> > > our
> > production system
> > >
> > >     >
> > >     >     > which does exactly this.
> > >     >
> > >     >     > We window by the event time, but trigger punctuate in
> > <punctuate interval>
> > >
> > >     >
> > >     >     > duration of system time, in the absence of an event
> > > crossing
> > the punctuate
> > >
> > >     >
> > >     >     > event time.
> > >     >
> > >     >     >
> > >     >
> > >     >     > We are using Kafka Stream for our Audit Trail, where
> > > we need
> > to output the
> > >
> > >     >
> > >     >     > event counts on each topic on each cluster aggregated
> > > over a
> > 1 minute
> > >
> > >     >
> > >     >     > window. We have to use event time to be able to cross
> > > check
> > the counts. But
> > >
> > >     >
> > >     >     > we need to trigger punctuate [aggregate event pushes]
> > > by
> > system time in the
> > >
> > >     >
> > >     >     > absence of events. Otherwise the event counts for
> > > unexpired
> > windows would
> > >
> > >     >
> > >     >     > be 0 which is bad.
> > >     >
> > >     >     >
> > >     >
> > >     >     > "Maybe a hybrid solution works: I window by event
> > > time but
> > trigger results
> > >
> > >     >
> > >     >     > by system time for windows that have updated? Not
> > > really sure
> > the details
> > >
> > >     >
> > >     >     > of making that work. Does that work? Are there
> > > concrete
> > examples where you
> > >
> > >     >
> > >     >     > actually want the current behavior?"
> > >     >
> > >     >     >
> > >     >
> > >     >     > --
> > >     >
> > >     >     > With Regards,
> > >     >
> > >     >     >
> > >     >
> > >     >     > Arun Mathew
> > >     >
> > >     >     > Yahoo! JAPAN Corporation
> > >     >
> > >     >     >
> > >     >
> > >     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
> > skyahead@gmail.com>
> > <sk...@gmail.com> wrote:
> > >
> > >     >
> > >     >     >
> > >     >
> > >     >     >> Hi Jay,
> > >     >
> > >     >     >>
> > >     >
> > >     >     >> The hybrid solution is exactly what I expect and
> > > need for
> > our use cases
> > >
> > >     >
> > >     >     >> when dealing with telecom data.
> > >     >
> > >     >     >>
> > >     >
> > >     >     >> Thanks
> > >     >
> > >     >     >> Tianji
> > >     >
> > >     >     >>
> > >     >
> > >     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
> > jay@confluent.io>
> > <ja...@confluent.io> wrote:
> > >
> > >     >
> > >     >     >>
> > >     >
> > >     >     >>> Hey guys,
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>> One thing I've always found super important for
> > > this kind
> > of design work
> > >
> > >     >
> > >     >     >> is
> > >     >
> > >     >     >>> to do a really good job of cataloging the landscape
> > > of use
> > cases and how
> > >
> > >     >
> > >     >     >>> prevalent each one is. By that I mean not just
> > > listing lots
> > of uses, but
> > >
> > >     >
> > >     >     >>> also grouping them into categories that
> > > functionally need
> > the same thing.
> > >
> > >     >
> > >     >     >>> In the absence of this it is very hard to reason
> > > about
> > design proposals.
> > >
> > >     >
> > >     >     >>> From the proposals so far I think we have a lot of
> > discussion around
> > >
> > >     >
> > >     >     >>> possible apis, but less around what the user needs
> > > for
> > different use
> > >
> > >     >
> > >     >     >> cases
> > >     >
> > >     >     >>> and how they would implement that using the api.
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>> Here is an example:
> > >     >
> > >     >     >>> You aggregate click and impression data for a
> > > reddit like
> > site. Every ten
> > >
> > >     >
> > >     >     >>> minutes you want to output a ranked list of the top
> > > 10
> > articles ranked by
> > >
> > >     >
> > >     >     >>> clicks/impressions for each geographical area. I
> > > want to be
> > able run this
> > >
> > >     >
> > >     >     >>> in steady state as well as rerun to regenerate
> > > results (or
> > catch up if it
> > >
> > >     >
> > >     >     >>> crashes).
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>> There are a couple of tricky things that seem to
> > > make this
> > hard with
> > >
> > >     >
> > >     >     >> either
> > >     >
> > >     >     >>> of the options proposed:
> > >     >
> > >     >     >>> 1. If I emit this data using event time I have the
> > > problem
> > described
> > >
> > >     >
> > >     >     >> where
> > >     >
> > >     >     >>> a geographical region with no new clicks or
> > > impressions
> > will fail to
> > >
> > >     >
> > >     >     >> output
> > >     >
> > >     >     >>> results.
> > >     >
> > >     >     >>> 2. If I emit this data using system time I have the
> > > problem
> > that when
> > >
> > >     >
> > >     >     >>> reprocessing data my window may not be ten minutes
> > > but 10
> > hours if my
> > >
> > >     >
> > >     >     >>> processing is very fast so it dramatically changes
> > > the
> > output.
> > >
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>> Maybe a hybrid solution works: I window by event
> > > time but
> > trigger results
> > >
> > >     >
> > >     >     >>> by system time for windows that have updated? Not
> > > really
> > sure the details
> > >
> > >     >
> > >     >     >>> of making that work. Does that work? Are there
> > > concrete
> > examples where
> > >
> > >     >
> > >     >     >> you
> > >     >
> > >     >     >>> actually want the current behavior?
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>> -Jay
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
> > arunmathew88@gmail.com> <ar...@gmail.com>
> > >
> > >     >
> > >     >     >>> wrote:
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>>> Hi All,
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> Thanks for the KIP. We were also in need of a
> > > mechanism to
> > trigger
> > >
> > >     >
> > >     >     >>>> punctuate in the absence of events.
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> As I described in [
> > >     >
> > >     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
> > >     >
> > >     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
> > >     >
> > >     >     >>>> plugin.system.issuetabpanels:comment-
> > > tabpanel#comment-
> > 15926036
> > >
> > >     >
> > >     >     >>>> ],
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>>    - Our approached involved using the event time
> > > by
> > default.
> > >
> > >     >
> > >     >     >>>>    - The method to check if there is any punctuate
> > > ready
> > in the
> > >
> > >     >
> > >     >     >>>>    PunctuationQueue is triggered via the any event
> > received by the
> > >
> > >     >
> > >     >     >> stream
> > >     >
> > >     >     >>>>    tread, or at the polling intervals in the
> > > absence of
> > any events.
> > >
> > >     >
> > >     >     >>>>    - When we create Punctuate objects (which
> > > contains the
> > next event
> > >
> > >     >
> > >     >     >> time
> > >     >
> > >     >     >>>>    for punctuation and interval), we also record
> > > the
> > creation time
> > >
> > >     >
> > >     >     >>> (system
> > >     >
> > >     >     >>>>    time).
> > >     >
> > >     >     >>>>    - While checking for maturity of Punctuate
> > > Schedule by
> > >     >
> > >     >     >> mayBePunctuate
> > >     >
> > >     >     >>>>    method, we also check if the system clock has
> > > elapsed
> > the punctuate
> > >
> > >     >
> > >     >     >>>>    interval since the schedule creation time.
> > >     >
> > >     >     >>>>    - In the absence of any event, or in the
> > > absence of any
> > event for
> > >
> > >     >
> > >     >     >> one
> > >     >
> > >     >     >>>>    topic in the partition group assigned to the
> > > stream
> > task, the system
> > >
> > >     >
> > >     >     >>>> time
> > >     >
> > >     >     >>>>    will elapse the interval and we trigger a
> > > punctuate
> > using the
> > >
> > >     >
> > >     >     >> expected
> > >     >
> > >     >     >>>>    punctuation event time.
> > >     >
> > >     >     >>>>    - we then create the next punctuation schedule
> > > as
> > punctuation event
> > >
> > >     >
> > >     >     >>> time
> > >     >
> > >     >     >>>>    + punctuation interval, [again recording the
> > > system
> > time of creation
> > >
> > >     >
> > >     >     >>> of
> > >     >
> > >     >     >>>> the
> > >     >
> > >     >     >>>>    schedule].
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> We call this a Hybrid Punctuate. Of course, this
> > > approach
> > has pros and
> > >
> > >     >
> > >     >     >>>> cons.
> > >     >
> > >     >     >>>> Pros
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>>    - Punctuates will happen in <punctuate
> > > interval> time
> > duration at
> > >
> > >     >
> > >     >     >> max
> > >     >
> > >     >     >>> in
> > >     >
> > >     >     >>>>    terms of system time.
> > >     >
> > >     >     >>>>    - The semantics as a whole continues to revolve
> > > around
> > event time.
> > >
> > >     >
> > >     >     >>>>    - We can use the old data [old timestamps] to
> > > rerun any
> > experiments
> > >
> > >     >
> > >     >     >> or
> > >     >
> > >     >     >>>>    tests.
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> Cons
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>>    - In case the  <punctuate interval> is not a
> > > time
> > duration [say
> > >
> > >     >
> > >     >     >>> logical
> > >     >
> > >     >     >>>>    time/event count], then the approach might not
> > > be
> > meaningful.
> > >
> > >     >
> > >     >     >>>>    - In case there is a case where we have to wait
> > > for an
> > actual event
> > >
> > >     >
> > >     >     >>> from
> > >     >
> > >     >     >>>>    a low event rate partition in the partition
> > > group, this
> > approach
> > >
> > >     >
> > >     >     >> will
> > >     >
> > >     >     >>>> jump
> > >     >
> > >     >     >>>>    the gun.
> > >     >
> > >     >     >>>>    - in case the event processing cannot catch up
> > > with the
> > event rate
> > >
> > >     >
> > >     >     >> and
> > >     >
> > >     >     >>>>    the expected timestamp events gets queued for
> > > long
> > time, this
> > >
> > >     >
> > >     >     >> approach
> > >     >
> > >     >     >>>>    might jump the gun.
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> I believe the above approach and discussion goes
> > > close to
> > the approach
> > >
> > >     >
> > >     >     >> A.
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> -----------
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> I like the idea of having an even count based
> > > punctuate.
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> -----------
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> I agree with the discussion around approach C,
> > > that we
> > should provide
> > >
> > >     >
> > >     >     >> the
> > >     >
> > >     >     >>>> user with the option to choose system time or
> > > event time
> > based
> > >
> > >     >
> > >     >     >>> punctuates.
> > >     >
> > >     >     >>>> But I believe that the user predominantly wants to
> > > use
> > event time while
> > >
> > >     >
> > >     >     >>> not
> > >     >
> > >     >     >>>> missing out on regular punctuates due to event
> > > delays or
> > event
> > >
> > >     >
> > >     >     >> absences.
> > >     >
> > >     >     >>>> Hence a complex punctuate option as Matthias
> > > mentioned
> > (quoted below)
> > >
> > >     >
> > >     >     >>> would
> > >     >
> > >     >     >>>> be most apt.
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> "- We might want to add "complex" schedules later
> > > on
> > (like, punctuate
> > >
> > >     >
> > >     >     >> on
> > >     >
> > >     >     >>>> every 10 seconds event-time or 60 seconds system-
> > > time
> > whatever comes
> > >
> > >     >
> > >     >     >>>> first)."
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> -----------
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> I think I read somewhere that Kafka Streams
> > > started with
> > System Time as
> > >
> > >     >
> > >     >     >>> the
> > >     >
> > >     >     >>>> punctuation standard, but was later changed to
> > > Event Time.
> > I guess
> > >
> > >     >
> > >     >     >> there
> > >     >
> > >     >     >>>> would be some good reason behind it. As Kafka
> > > Streams want
> > to evolve
> > >
> > >     >
> > >     >     >> more
> > >     >
> > >     >     >>>> on the Stream Processing front, I believe the
> > > emphasis on
> > event time
> > >
> > >     >
> > >     >     >>> would
> > >     >
> > >     >     >>>> remain quite strong.
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> With Regards,
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> Arun Mathew
> > >     >
> > >     >     >>>> Yahoo! JAPAN Corporation, Tokyo
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
> > tobecker@tivo.com> <to...@tivo.com>
> > >
> > >     >
> > >     >     >> wrote:
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>>> Yeah I like PuncutationType much better; I just
> > > threw
> > Time out there
> > >
> > >     >
> > >     >     >>>>> more as a strawman than an actual suggestion ;) I
> > > still
> > think it's
> > >
> > >     >
> > >     >     >>>>> worth considering what this buys us over an
> > > additional
> > callback. I
> > >
> > >     >
> > >     >     >>>>> foresee a number of punctuate implementations
> > > following
> > this pattern:
> > >
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>> public void punctuate(PunctuationType type) {
> > >     >
> > >     >     >>>>>     switch (type) {
> > >     >
> > >     >     >>>>>         case EVENT_TIME:
> > >     >
> > >     >     >>>>>             methodA();
> > >     >
> > >     >     >>>>>             break;
> > >     >
> > >     >     >>>>>         case SYSTEM_TIME:
> > >     >
> > >     >     >>>>>             methodB();
> > >     >
> > >     >     >>>>>             break;
> > >     >
> > >     >     >>>>>     }
> > >     >
> > >     >     >>>>> }
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>> I guess one advantage of this approach is we
> > > could add
> > additional
> > >
> > >     >
> > >     >     >>>>> punctuation types later in a backwards compatible
> > > way
> > (like event
> > >
> > >     >
> > >     >     >> count
> > >     >
> > >     >     >>>>> as you mentioned).
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>> -Tommy
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J.
> > > Sax wrote:
> > >     >
> > >     >     >>>>>> That sounds promising.
> > >     >
> > >     >     >>>>>>
> > >     >
> > >     >     >>>>>> I am just wondering if `Time` is the best name.
> > > Maybe we
> > want to
> > >
> > >     >
> > >     >     >> add
> > >     >
> > >     >     >>>>>> other non-time based punctuations at some point
> > > later. I
> > would
> > >
> > >     >
> > >     >     >>>>>> suggest
> > >     >
> > >     >     >>>>>>
> > >     >
> > >     >     >>>>>> enum PunctuationType {
> > >     >
> > >     >     >>>>>>   EVENT_TIME,
> > >     >
> > >     >     >>>>>>   SYSTEM_TIME,
> > >     >
> > >     >     >>>>>> }
> > >     >
> > >     >     >>>>>>
> > >     >
> > >     >     >>>>>> or similar. Just to keep the door open -- it's
> > > easier to
> > add new
> > >
> > >     >
> > >     >     >>>>>> stuff
> > >     >
> > >     >     >>>>>> if the name is more generic.
> > >     >
> > >     >     >>>>>>
> > >     >
> > >     >     >>>>>>
> > >     >
> > >     >     >>>>>> -Matthias
> > >     >
> > >     >     >>>>>>
> > >     >
> > >     >     >>>>>>
> > >     >
> > >     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>> I agree that the framework providing and
> > > managing the
> > notion of
> > >
> > >     >
> > >     >     >>>>>>> stream
> > >     >
> > >     >     >>>>>>> time is valuable and not something we would
> > > want to
> > delegate to
> > >
> > >     >
> > >     >     >> the
> > >     >
> > >     >     >>>>>>> tasks. I'm not entirely convinced that a
> > > separate
> > callback
> > >
> > >     >
> > >     >     >> (option
> > >     >
> > >     >     >>>>>>> C)
> > >     >
> > >     >     >>>>>>> is that messy (it could just be a default
> > > method with
> > an empty
> > >
> > >     >
> > >     >     >>>>>>> implementation), but if we wanted a single API
> > > to
> > handle both
> > >
> > >     >
> > >     >     >>>>>>> cases,
> > >     >
> > >     >     >>>>>>> how about something like the following?
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>> enum Time {
> > >     >
> > >     >     >>>>>>>    STREAM,
> > >     >
> > >     >     >>>>>>>    CLOCK
> > >     >
> > >     >     >>>>>>> }
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>> Then on ProcessorContext:
> > >     >
> > >     >     >>>>>>> context.schedule(Time time, long interval)  //
> > > We could
> > allow
> > >
> > >     >
> > >     >     >> this
> > >     >
> > >     >     >>>>>>> to
> > >     >
> > >     >     >>>>>>> be called once for each value of time to mix
> > approaches.
> > >
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>> Then the Processor API becomes:
> > >     >
> > >     >     >>>>>>> punctuate(Time time) // time here denotes which
> > schedule resulted
> > >
> > >     >
> > >     >     >>>>>>> in
> > >     >
> > >     >     >>>>>>> this call.
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>> Thoughts?
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J.
> > > Sax
> > wrote:
> > >
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> Thanks a lot for the KIP Michal,
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> I was thinking about the four options you
> > > proposed in
> > more
> > >
> > >     >
> > >     >     >>>>>>>> details
> > >     >
> > >     >     >>>>>>>> and
> > >     >
> > >     >     >>>>>>>> this are my thoughts:
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> (A) You argue, that users can still
> > > "punctuate" on
> > event-time
> > >
> > >     >
> > >     >     >> via
> > >     >
> > >     >     >>>>>>>> process(), but I am not sure if this is
> > > possible.
> > Note, that
> > >
> > >     >
> > >     >     >>>>>>>> users
> > >     >
> > >     >     >>>>>>>> only
> > >     >
> > >     >     >>>>>>>> get record timestamps via context.timestamp().
> > > Thus,
> > users
> > >
> > >     >
> > >     >     >> would
> > >     >
> > >     >     >>>>>>>> need
> > >     >
> > >     >     >>>>>>>> to
> > >     >
> > >     >     >>>>>>>> track the time progress per partition (based
> > > on the
> > partitions
> > >
> > >     >
> > >     >     >>>>>>>> they
> > >     >
> > >     >     >>>>>>>> obverse via context.partition(). (This alone
> > > puts a
> > huge burden
> > >
> > >     >
> > >     >     >>>>>>>> on
> > >     >
> > >     >     >>>>>>>> the
> > >     >
> > >     >     >>>>>>>> user by itself.) However, users are not
> > > notified at
> > startup
> > >
> > >     >
> > >     >     >> what
> > >     >
> > >     >     >>>>>>>> partitions are assigned, and user are not
> > > notified
> > when
> > >
> > >     >
> > >     >     >>>>>>>> partitions
> > >     >
> > >     >     >>>>>>>> get
> > >     >
> > >     >     >>>>>>>> revoked. Because this information is not
> > > available,
> > it's not
> > >
> > >     >
> > >     >     >>>>>>>> possible
> > >     >
> > >     >     >>>>>>>> to
> > >     >
> > >     >     >>>>>>>> "manually advance" stream-time, and thus
> > > event-time
> > punctuation
> > >
> > >     >
> > >     >     >>>>>>>> within
> > >     >
> > >     >     >>>>>>>> process() seems not to be possible -- or do
> > > you see a
> > way to
> > >
> > >     >
> > >     >     >> get
> > >     >
> > >     >     >>>>>>>> it
> > >     >
> > >     >     >>>>>>>> done? And even if, it might still be too
> > > clumsy to
> > use.
> > >
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> (B) This does not allow to mix both
> > > approaches, thus
> > limiting
> > >
> > >     >
> > >     >     >>>>>>>> what
> > >     >
> > >     >     >>>>>>>> users
> > >     >
> > >     >     >>>>>>>> can do.
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> (C) This should give all flexibility we need.
> > > However,
> > just
> > >
> > >     >
> > >     >     >>>>>>>> adding
> > >     >
> > >     >     >>>>>>>> one
> > >     >
> > >     >     >>>>>>>> more method seems to be a solution that is too
> > > simple
> > (cf my
> > >
> > >     >
> > >     >     >>>>>>>> comments
> > >     >
> > >     >     >>>>>>>> below).
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> (D) This might be hard to use. Also, I am not
> > > sure how
> > a user
> > >
> > >     >
> > >     >     >>>>>>>> could
> > >     >
> > >     >     >>>>>>>> enable system-time and event-time punctuation
> > > in
> > parallel.
> > >
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> Overall options (C) seems to be the most
> > > promising
> > approach to
> > >
> > >     >
> > >     >     >>>>>>>> me.
> > >     >
> > >     >     >>>>>>>> Because I also favor a clean API, we might
> > > keep
> > current
> > >
> > >     >
> > >     >     >>>>>>>> punctuate()
> > >     >
> > >     >     >>>>>>>> as-is, but deprecate it -- so we can remove it
> > > at some
> > later
> > >
> > >     >
> > >     >     >>>>>>>> point
> > >     >
> > >     >     >>>>>>>> when
> > >     >
> > >     >     >>>>>>>> people use the "new punctuate API".
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> Couple of follow up questions:
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> - I am wondering, if we should have two
> > > callback
> > methods or
> > >
> > >     >
> > >     >     >> just
> > >     >
> > >     >     >>>>>>>> one
> > >     >
> > >     >     >>>>>>>> (ie, a unified for system and event time
> > > punctuation
> > or one for
> > >
> > >     >
> > >     >     >>>>>>>> each?).
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> - If we have one, how can the user figure out,
> > > which
> > condition
> > >
> > >     >
> > >     >     >>>>>>>> did
> > >     >
> > >     >     >>>>>>>> trigger?
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> - How would the API look like, for registering
> > different
> > >
> > >     >
> > >     >     >>>>>>>> punctuate
> > >     >
> > >     >     >>>>>>>> schedules? The "type" must be somehow defined?
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> - We might want to add "complex" schedules
> > > later on
> > (like,
> > >
> > >     >
> > >     >     >>>>>>>> punctuate
> > >     >
> > >     >     >>>>>>>> on
> > >     >
> > >     >     >>>>>>>> every 10 seconds event-time or 60 seconds
> > > system-time
> > whatever
> > >
> > >     >
> > >     >     >>>>>>>> comes
> > >     >
> > >     >     >>>>>>>> first). I don't say we should add this right
> > > away, but
> > we might
> > >
> > >     >
> > >     >     >>>>>>>> want
> > >     >
> > >     >     >>>>>>>> to
> > >     >
> > >     >     >>>>>>>> define the API in a way, that it allows
> > > extensions
> > like this
> > >
> > >     >
> > >     >     >>>>>>>> later
> > >     >
> > >     >     >>>>>>>> on,
> > >     >
> > >     >     >>>>>>>> without redesigning the API (ie, the API
> > > should be
> > designed
> > >
> > >     >
> > >     >     >>>>>>>> extensible)
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> - Did you ever consider count-based
> > > punctuation?
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> I understand, that you would like to solve a
> > > simple
> > problem,
> > >
> > >     >
> > >     >     >> but
> > >     >
> > >     >     >>>>>>>> we
> > >     >
> > >     >     >>>>>>>> learned from the past, that just "adding some
> > > API"
> > quickly
> > >
> > >     >
> > >     >     >> leads
> > >     >
> > >     >     >>>>>>>> to a
> > >     >
> > >     >     >>>>>>>> not very well defined API that needs time
> > > consuming
> > clean up
> > >
> > >     >
> > >     >     >>>>>>>> later on
> > >     >
> > >     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a
> > > holistic
> > >     >
> > >     >     >>>>>>>> punctuation
> > >     >
> > >     >     >>>>>>>> KIP
> > >     >
> > >     >     >>>>>>>> with this from the beginning on to avoid later
> > > painful
> > >     >
> > >     >     >> redesign.
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> -Matthias
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>>
> > >     >
> > >     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>> Thanks Thomas,
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>> I'm also wary of changing the existing
> > > semantics of
> > >     >
> > >     >     >> punctuate,
> > >     >
> > >     >     >>>>>>>>> for
> > >     >
> > >     >     >>>>>>>>> backward compatibility reasons, although I
> > > like the
> > >     >
> > >     >     >> conceptual
> > >     >
> > >     >     >>>>>>>>> simplicity of that option.
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>> Adding a new method to me feels safer but, in
> > > a way,
> > uglier.
> > >
> > >     >
> > >     >     >> I
> > >     >
> > >     >     >>>>>>>>> added
> > >     >
> > >     >     >>>>>>>>> this to the KIP now as option (C).
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>> The TimestampExtractor mechanism is actually
> > > more
> > flexible,
> > >
> > >     >
> > >     >     >> as
> > >     >
> > >     >     >>>>>>>>> it
> > >     >
> > >     >     >>>>>>>>> allows
> > >     >
> > >     >     >>>>>>>>> you to return any value, you're not limited
> > > to event
> > time or
> > >
> > >     >
> > >     >     >>>>>>>>> system
> > >     >
> > >     >     >>>>>>>>> time
> > >     >
> > >     >     >>>>>>>>> (although I don't see an actual use case
> > > where you
> > might need
> > >
> > >     >
> > >     >     >>>>>>>>> anything
> > >     >
> > >     >     >>>>>>>>> else then those two). Hence I also proposed
> > > the
> > option to
> > >
> > >     >
> > >     >     >> allow
> > >     >
> > >     >     >>>>>>>>> users
> > >     >
> > >     >     >>>>>>>>> to, effectively, decide what "stream time" is
> > > for
> > them given
> > >
> > >     >
> > >     >     >>>>>>>>> the
> > >     >
> > >     >     >>>>>>>>> presence or absence of messages, much like
> > > they can
> > decide
> > >
> > >     >
> > >     >     >> what
> > >     >
> > >     >     >>>>>>>>> msg
> > >     >
> > >     >     >>>>>>>>> time
> > >     >
> > >     >     >>>>>>>>> means for them using the TimestampExtractor.
> > > What do
> > you
> > >
> > >     >
> > >     >     >> think
> > >     >
> > >     >     >>>>>>>>> about
> > >     >
> > >     >     >>>>>>>>> that? This is probably most flexible but also
> > > most
> > >     >
> > >     >     >> complicated.
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>> All comments appreciated.
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>> Cheers,
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>> Michal
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>
> > >     >
> > >     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
> > >     >
> > >     >     >>>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>> Although I fully agree we need a way to
> > > trigger
> > periodic
> > >
> > >     >
> > >     >     >>>>>>>>>> processing
> > >     >
> > >     >     >>>>>>>>>> that is independent from whether and when
> > > messages
> > arrive,
> > >
> > >     >
> > >     >     >>>>>>>>>> I'm
> > >     >
> > >     >     >>>>>>>>>> not sure
> > >     >
> > >     >     >>>>>>>>>> I like the idea of changing the existing
> > > semantics
> > across
> > >
> > >     >
> > >     >     >> the
> > >     >
> > >     >     >>>>>>>>>> board.
> > >     >
> > >     >     >>>>>>>>>> What if we added an additional callback to
> > > Processor
> > that
> > >
> > >     >
> > >     >     >> can
> > >     >
> > >     >     >>>>>>>>>> be
> > >     >
> > >     >     >>>>>>>>>> scheduled similarly to punctuate() but was
> > > always
> > called at
> > >
> > >     >
> > >     >     >>>>>>>>>> fixed, wall
> > >     >
> > >     >     >>>>>>>>>> clock based intervals? This way you wouldn't
> > > have to
> > give
> > >
> > >     >
> > >     >     >> up
> > >     >
> > >     >     >>>>>>>>>> the
> > >     >
> > >     >     >>>>>>>>>> notion
> > >     >
> > >     >     >>>>>>>>>> of stream time to be able to do periodic
> > > processing.
> > >     >
> > >     >     >>>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal
> > > Borowiecki
> > wrote:
> > >
> > >     >
> > >     >     >>>>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>>> Hi all,
> > >     >
> > >     >     >>>>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
> > punctuate
> > >
> > >     >
> > >     >     >>>>>>>>>>> semantics
> > >     >
> > >     >     >>>>>>>>>>> <https://cwiki.apache.org/
> > confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
> > confluence/display/KAFKA/KIP->
> > >
> > >     >
> > >     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KI
> > > P->>>
> > 138%
> > >
> > >     >
> > >     >     >>>>>>>>>>> 3A+C
> > >     >
> > >     >     >>>>>>>>>>> hange+
> > >     >
> > >     >     >>>>>>>>>>> punctuate+semantics>
> > >     >
> > >     >     >>>>>>>>>>> .
> > >     >
> > >     >     >>>>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>>> Appreciating there can be different views
> > > on
> > system-time
> > >
> > >     >
> > >     >     >> vs
> > >     >
> > >     >     >>>>>>>>>>> event-
> > >     >
> > >     >     >>>>>>>>>>> time
> > >     >
> > >     >     >>>>>>>>>>> semantics for punctuation depending on use-
> > > case and
> > the
> > >
> > >     >
> > >     >     >>>>>>>>>>> importance of
> > >     >
> > >     >     >>>>>>>>>>> backwards compatibility of any such change,
> > > I've
> > left it
> > >
> > >     >
> > >     >     >>>>>>>>>>> quite
> > >     >
> > >     >     >>>>>>>>>>> open
> > >     >
> > >     >     >>>>>>>>>>> and
> > >     >
> > >     >     >>>>>>>>>>> hope to fill in more info as the discussion
> > progresses.
> > >
> > >     >
> > >     >     >>>>>>>>>>>
> > >     >
> > >     >     >>>>>>>>>>> Thanks,
> > >     >
> > >     >     >>>>>>>>>>> Michal
> > >     >
> > >     >     >>>>>>> --
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>>     Tommy Becker
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>>     Senior Software Engineer
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>>     tivo.com
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>> ________________________________
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>>>> This email and any attachments may contain
> > > confidential
> > and
> > >
> > >     >
> > >     >     >>>>>>> privileged material for the sole use of the
> > > intended
> > recipient.
> > >
> > >     >
> > >     >     >> Any
> > >     >
> > >     >     >>>>>>> review, copying, or distribution of this email
> > > (or any
> > >     >
> > >     >     >> attachments)
> > >     >
> > >     >     >>>>>>> by others is prohibited. If you are not the
> > > intended
> > recipient,
> > >
> > >     >
> > >     >     >>>>>>> please contact the sender immediately and
> > > permanently
> > delete this
> > >
> > >     >
> > >     >     >>>>>>> email and any attachments. No employee or agent
> > > of TiVo
> > Inc. is
> > >
> > >     >
> > >     >     >>>>>>> authorized to conclude any binding agreement on
> > > behalf
> > of TiVo
> > >
> > >     >
> > >     >     >> Inc.
> > >     >
> > >     >     >>>>>>> by email. Binding agreements with TiVo Inc. may
> > > only be
> > made by a
> > >
> > >     >
> > >     >     >>>>>>> signed written agreement.
> > >     >
> > >     >     >>>>>>>
> > >     >
> > >     >     >>>>> --
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>>     Tommy Becker
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>>     Senior Software Engineer
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>>     tivo.com
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>> ________________________________
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>> This email and any attachments may contain
> > > confidential
> > and
> > >
> > >     >
> > >     >     >> privileged
> > >     >
> > >     >     >>>>> material for the sole use of the intended
> > > recipient. Any
> > review,
> > >
> > >     >
> > >     >     >>> copying,
> > >     >
> > >     >     >>>>> or distribution of this email (or any
> > > attachments) by
> > others is
> > >
> > >     >
> > >     >     >>>> prohibited.
> > >     >
> > >     >     >>>>> If you are not the intended recipient, please
> > > contact the
> > sender
> > >
> > >     >
> > >     >     >>>>> immediately and permanently delete this email and
> > > any
> > attachments. No
> > >
> > >     >
> > >     >     >>>>> employee or agent of TiVo Inc. is authorized to
> > > conclude
> > any binding
> > >
> > >     >
> > >     >     >>>>> agreement on behalf of TiVo Inc. by email.
> > > Binding
> > agreements with
> > >
> > >     >
> > >     >     >> TiVo
> > >     >
> > >     >     >>>>> Inc. may only be made by a signed written
> > > agreement.
> > >     >
> > >     >     >>>>>
> > >     >
> > >     >     >>>>
> > >     >
> > >     >     >>>
> > >     >
> > >     >     >>
> > >     >
> > >     >     >
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >
> > >     >
> > >     > --
> > >     >
> > >     > <http://www.openbet.com/>
> > >     >
> > >     > *Michal Borowiecki*
> > >     >
> > >     > *Senior Software Engineer L4*
> > >     >
> > >     > *T: *
> > >     >
> > >     > +44 208 742 1600 <+44%2020%208742%201600>
> > >     >
> > >     > +44 203 249 8448 <+44%2020%203249%208448>
> > >     >
> > >     >
> > >     >
> > >     > *E: *
> > >     >
> > >     > michal.borowiecki@openbet.com
> > >     >
> > >     > *W: *
> > >     >
> > >     > www.openbet.com
> > >     >
> > >     > *OpenBet Ltd*
> > >     >
> > >     > Chiswick Park Building 9
> > >     >
> > >     > 566 Chiswick High Rd
> > >     >
> > >     > London
> > >     >
> > >     > W4 5XT
> > >     >
> > >     > UK
> > >     >
> > >     > <https://www.openbet.com/email_promo>
> > >     >
> > >     >
> > >     >
> > >     > This message is confidential and intended only for the
> > > addressee.
> > If you
> > >
> > >     > have received this message in error, please immediately
> > > notify the
> > >     > postmaster@openbet.com and delete it from your system as
> > > well as
> > any
> > >
> > >     > copies. The content of e-mails as well as traffic data may
> > > be
> > monitored by
> > >
> > >     > OpenBet for employment and security purposes. To protect
> > > the
> > environment
> > >
> > >     > please do not print this e-mail unless necessary. OpenBet
> > > Ltd.
> > Registered
> > >
> > >     > Office: Chiswick Park Building 9, 566 Chiswick High Road,
> > > London,
> > W4 5XT,
> > >
> > >     > United Kingdom. A company registered in England and Wales.
> > Registered no.
> > >
> > >     > 3134634. VAT no. GB927523612
> > >     >
> > >     >
> > >     >
> > >
> > >
--


    Tommy Becker

    Senior Software Engineer

    O +1 919.460.4747

    tivo.com


________________________________

This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michael Noll <mi...@confluent.io>.
I apologize for the longer email below.  To my defense, it started out much
shorter. :-)  Also, to be super-clear, I am intentionally playing devil's
advocate for a number of arguments brought forth in order to help improve
this KIP -- I am not implying I necessarily disagree with the arguments.

That aside, here are some further thoughts.

First, there are (at least?) two categories for actions/behavior you invoke
via punctuate():

1. For internal housekeeping of your Processor or Transformer (e.g., to
periodically commit to a custom store, to do metrics/logging).  Here, the
impact of punctuate is typically not observable by other processing nodes
in the topology.
2. For controlling the emit frequency of downstream records.  Here, the
punctuate is all about being observable by downstream processing nodes.

A few releases back, we introduced record caches (DSL) and state store
caches (Processor API) in KIP-63.  Here, we addressed a concern relating to
(2) where some users needed to control -- here: limit -- the downstream
output rate of Kafka Streams because the downstream systems/apps would not
be able to keep up with the upstream output rate (Kafka scalability > their
scalability).  The argument for KIP-63, which notably did not introduce a
"trigger" API, was that such an interaction with downstream systems is an
operational concern;  it should not impact the processing *logic* of your
application, and thus we didn't want to complicate the Kafka Streams API,
especially not the declarative DSL, with such operational concerns.

This KIP's discussion on `punctuate()` takes us back in time (<-- sorry, I
couldn't resist to not make this pun :-P).  As a meta-comment, I am
observing that our conversation is moving more and more into the direction
of explicit "triggers" because, so far, I have seen only motivations for
use cases in category (2), but none yet for (1)?  For example, some
comments voiced here are about sth like "IF stream-time didn't trigger
punctuate, THEN trigger punctuate based on processing-time".  Do we want
this, and if so, for which use cases and benefits?  Also, on a related
note, whatever we are discussing here will impact state store caches
(Processor API) and perhaps also impact record caches (DSL), thus we should
clarify any such impact here.

Switching topics slightly.

Jay wrote:
> One thing I've always found super important for this kind of design work
> is to do a really good job of cataloging the landscape of use cases and
> how prevalent each one is.

+1 to this, as others have already said.

Here, let me highlight -- just in case -- that when we talked about
windowing use cases in the recent emails, the Processor API (where
`punctuate` resides) does not have any notion of windowing at all.  If you
want to do windowing *in the Processor API*, you must do so manually in
combination with window stores.  For this reason I'd suggest to discuss use
cases not just in general, but also in view of how you'd do so in the
Processor API vs. in the DSL.  Right now, changing/improving `punctuate`
does not impact the DSL at all, unless we add new functionality to it.

Jay wrote in his strawman example:
> You aggregate click and impression data for a reddit like site. Every ten
> minutes you want to output a ranked list of the top 10 articles ranked by
> clicks/impressions for each geographical area. I want to be able run this
> in steady state as well as rerun to regenerate results (or catch up if it
> crashes).

This is a good example for more than the obvious reason:  In KIP-63, we
argued that the reason for saying "every ten minutes" above is not
necessarily about because you want to output data *exactly* after ten
minutes, but that you want to perform an aggregation based on 10-minute
windows of input data; i.e., the point is about specifying the input for
your aggregation, not or less about when the results of the aggregation
should be send downstream.  To take an extreme example, you could disable
record caches and let your app output a downstream update for every
incoming input record.  If the last input record was from at minute 7 of 10
(for a 10-min window), then what your app would output at minute 10 would
be identical to what it had already emitted at minute 7 earlier anyways.
This is particularly true when we take late-arriving data into account:  if
a late record arrived at minute 13, your app would (by default) send a new
update downstream, even though the "original" 10 minutes have already
passed.

Jay wrote...:
> There are a couple of tricky things that seem to make this hard with
either
> of the options proposed:
> 1. If I emit this data using event time I have the problem described where
> a geographical region with no new clicks or impressions will fail to
output
> results.

...and Arun Mathew wrote:

> We window by the event time, but trigger punctuate in <punctuate interval>
> duration of system time, in the absence of an event crossing the punctuate
> event time.

So, given what I wrote above about the status quo and what you can already
do with it, is the concern that the state store cache doesn't give you
*direct* control over "forcing an output after no later than X seconds [of
processing-time]" but only indirect control through a cache size?  (Note
that I am not dismissing the claims why this might be helpful.)

Arun Mathew wrote:
> We are using Kafka Stream for our Audit Trail, where we need to output the
> event counts on each topic on each cluster aggregated over a 1 minute
> window. We have to use event time to be able to cross check the counts.
But
> we need to trigger punctuate [aggregate event pushes] by system time in
the
> absence of events. Otherwise the event counts for unexpired windows would
> be 0 which is bad.

Isn't the latter -- "count would be 0" -- the problem between the absence
of output vs. an output of 0, similar to the use of `Option[T]` in Scala
and the difference between `None` and `Some(0)`?  That is, isn't the root
cause that the downstream system interprets the absence of output in a
particular way ("No output after 1 minute = I consider the output to be
0.")?  Arguably, you could also adapt the downstream system (if possible)
to correctly handle the difference between absence of output vs. output of
0.  I am not implying that we shouldn't care about such a use case, but
want to understand the motivation better. :-)

Also, to add some perspective, in some related discussions we talked about
how a Kafka Streams application should not worry or not be coupled
unnecessarily with such interpretation specifics in a downstream system's
behavior.  After all, tomorrow your app's output might be consumed by more
than just this one downstream system.  Arguably, Kafka Connect rather than
Kafka Streams might be the best tool to link the universes of Kafka and
downstream systems, including helping to reconcile the differences in how
these systems interpret changes, updates, late-arriving data, etc.  Kafka
Connect would allow you to decouple the Kafka Streams app's logical
processing from the specifics of downstream systems, thanks to specific
sink connectors (e.g. for Elastic, Cassandra, MySQL, S3, etc).  Would this
decoupling with Kafka Connect help here?  (And if the answer is "Yes, but
it's currently awkward to use Connect for this", this might be a problem we
can solve, too.)

Switching topics slightly again.

Thomas wrote:
> I'm not entirely convinced that a separate callback (option C)
> is that messy (it could just be a default method with an empty
> implementation), but if we wanted a single API to handle both cases,
> how about something like the following?
>
> enum Time {
>    STREAM,
>    CLOCK
> }

Yeah, I am on the fence here, too.  If we use the 1-method approach, then
whatever the user is doing inside this method is a black box to Kafka
Streams (similar to how we have no idea what the user does inside a
`foreach` -- if the function passed to `foreach` writes to external
systems, then Kafka Streams is totally unaware of the fact).  We won't
know, for example, if the stream-time action has a smaller "trigger"
frequency than the processing-time action.  Or, we won't know whether the
user custom-codes a "not later than" trigger logic ("Do X every 1-minute of
stream-time or 1-minute of processing-time, whichever comes first").  That
said, I am not certain yet whether we would need such knowledge because,
when using the Processor API, most of the work and decisions must be done
by the user anyways.  It would matter though if the concept of "triggers"
were to bubble up into the DSL because in the DSL the management of
windowing, window stores, etc. must be done automatically by Kafka Streams.

[In any case, btw, we have the corner case where the user configured the
stream-time to be processing-time (e.g. via wall-clock timestamp
extractor), at which point both punctuate variants are based on the same
time semantics / timeline.]

Again, I apologize for the wall of text.  Congratulations if you made it
this far. :-)

More than happy to hear your thoughts!
Michael

On Tue, Apr 11, 2017 at 1:12 AM, Arun Mathew <ar...@gmail.com> wrote:

> Thanks Matthias.
> Sure, will correct it right away.
>
> On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> wrote:
>
> Thanks for preparing this page!
>
> About terminology:
>
> You introduce the term "event time" -- but we should call this "stream
> time" -- "stream time" is whatever TimestampExtractor returns and this
> could be event time, ingestion time, or processing/wall-clock time.
>
> Does this make sense to you?
>
>
>
> -Matthias
>
>
> On 4/10/17 4:58 AM, Arun Mathew wrote:
> > Thanks Ewen.
> >
> > @Michal, @all, I have created a child page to start the Use Cases
> discussion [https://cwiki.apache.org/confluence/display/KAFKA/
> Punctuate+Use+Cases]. Please go through it and give your comments.
> >
> > @Tianji, Sorry for the delay. I am trying to make the patch public.
> >
> > --
> > Arun Mathew
> >
> > On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> wrote:
> >
> >     Arun,
> >
> >     I've given you permission to edit the wiki. Let me know if you run
> into any
> >     issues.
> >
> >     -Ewen
> >
> >     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <am...@yahoo-corp.jp>
> wrote:
> >
> >     > Thanks Michal. I don’t have the access yet [arunmathew88]. Should I
> be
> >     > sending a separate mail for this?
> >     >
> >     > I thought one of the person following this thread would be able to
> give me
> >     > access.
> >     >
> >     >
> >     >
> >     > *From: *Michal Borowiecki <mi...@openbet.com>
> >     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> >     > *Date: *Friday, April 7, 2017 at 17:16
> >     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> >     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
> >     >
> >     >
> >     >
> >     > Hi Arun,
> >     >
> >     > I was thinking along the same lines as you, listing the use cases
> on the
> >     > wiki, but didn't find time to get around doing that yet.
> >     > Don't mind if you do it if you have access now.
> >     > I was thinking it would be nice if, once we have the use cases
> listed,
> >     > people could use likes to up-vote the use cases similar to what
> they're
> >     > working on.
> >     >
> >     > I should have a bit more time to action this in the next few days,
> but
> >     > happy for you to do it if you can beat me to it ;-)
> >     >
> >     > Cheers,
> >     > Michal
> >     >
> >     > On 07/04/17 04:39, Arun Mathew wrote:
> >     >
> >     > Sure, Thanks Matthias. My id is [arunmathew88].
> >     >
> >     >
> >     >
> >     > Of course. I was thinking of a subpage where people can
> collaborate.
> >     >
> >     >
> >     >
> >     > Will do as per Michael’s suggestion.
> >     >
> >     >
> >     >
> >     > Regards,
> >     >
> >     > Arun Mathew
> >     >
> >     >
> >     >
> >     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <
> matthias@confluent.io> wrote:
> >     >
> >     >
> >     >
> >     >     Please share your Wiki-ID and a committer can give you write
> access.
> >     >
> >     >
> >     >
> >     >     Btw: as you did not initiate the KIP, you should not change the
> KIP
> >     >
> >     >     without the permission of the original author -- in this case
> Michael.
> >     >
> >     >
> >     >
> >     >     So you might also just share your thought over the mailing list
> and
> >     >
> >     >     Michael can update the KIP page. Or, as an alternative, just
> create a
> >     >
> >     >     subpage for the KIP page.
> >     >
> >     >
> >     >
> >     >     @Michael: WDYT?
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >     -Matthias
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
> >     >
> >     >     > Hi Jay,
> >     >
> >     >     >           Thanks for the advise, I would like to list down
> the use cases as
> >     >
> >     >     > per your suggestion. But it seems I don't have write
> permission to the
> >     >
> >     >     > Apache Kafka Confluent Space. Whom shall I request for it?
> >     >
> >     >     >
> >     >
> >     >     > Regarding your last question. We are using a patch in our
> production system
> >     >
> >     >     > which does exactly this.
> >     >
> >     >     > We window by the event time, but trigger punctuate in
> <punctuate interval>
> >     >
> >     >     > duration of system time, in the absence of an event crossing
> the punctuate
> >     >
> >     >     > event time.
> >     >
> >     >     >
> >     >
> >     >     > We are using Kafka Stream for our Audit Trail, where we need
> to output the
> >     >
> >     >     > event counts on each topic on each cluster aggregated over a
> 1 minute
> >     >
> >     >     > window. We have to use event time to be able to cross check
> the counts. But
> >     >
> >     >     > we need to trigger punctuate [aggregate event pushes] by
> system time in the
> >     >
> >     >     > absence of events. Otherwise the event counts for unexpired
> windows would
> >     >
> >     >     > be 0 which is bad.
> >     >
> >     >     >
> >     >
> >     >     > "Maybe a hybrid solution works: I window by event time but
> trigger results
> >     >
> >     >     > by system time for windows that have updated? Not really sure
> the details
> >     >
> >     >     > of making that work. Does that work? Are there concrete
> examples where you
> >     >
> >     >     > actually want the current behavior?"
> >     >
> >     >     >
> >     >
> >     >     > --
> >     >
> >     >     > With Regards,
> >     >
> >     >     >
> >     >
> >     >     > Arun Mathew
> >     >
> >     >     > Yahoo! JAPAN Corporation
> >     >
> >     >     >
> >     >
> >     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <
> skyahead@gmail.com>
> <sk...@gmail.com> wrote:
> >     >
> >     >     >
> >     >
> >     >     >> Hi Jay,
> >     >
> >     >     >>
> >     >
> >     >     >> The hybrid solution is exactly what I expect and need for
> our use cases
> >     >
> >     >     >> when dealing with telecom data.
> >     >
> >     >     >>
> >     >
> >     >     >> Thanks
> >     >
> >     >     >> Tianji
> >     >
> >     >     >>
> >     >
> >     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <
> jay@confluent.io>
> <ja...@confluent.io> wrote:
> >     >
> >     >     >>
> >     >
> >     >     >>> Hey guys,
> >     >
> >     >     >>>
> >     >
> >     >     >>> One thing I've always found super important for this kind
> of design work
> >     >
> >     >     >> is
> >     >
> >     >     >>> to do a really good job of cataloging the landscape of use
> cases and how
> >     >
> >     >     >>> prevalent each one is. By that I mean not just listing lots
> of uses, but
> >     >
> >     >     >>> also grouping them into categories that functionally need
> the same thing.
> >     >
> >     >     >>> In the absence of this it is very hard to reason about
> design proposals.
> >     >
> >     >     >>> From the proposals so far I think we have a lot of
> discussion around
> >     >
> >     >     >>> possible apis, but less around what the user needs for
> different use
> >     >
> >     >     >> cases
> >     >
> >     >     >>> and how they would implement that using the api.
> >     >
> >     >     >>>
> >     >
> >     >     >>> Here is an example:
> >     >
> >     >     >>> You aggregate click and impression data for a reddit like
> site. Every ten
> >     >
> >     >     >>> minutes you want to output a ranked list of the top 10
> articles ranked by
> >     >
> >     >     >>> clicks/impressions for each geographical area. I want to be
> able run this
> >     >
> >     >     >>> in steady state as well as rerun to regenerate results (or
> catch up if it
> >     >
> >     >     >>> crashes).
> >     >
> >     >     >>>
> >     >
> >     >     >>> There are a couple of tricky things that seem to make this
> hard with
> >     >
> >     >     >> either
> >     >
> >     >     >>> of the options proposed:
> >     >
> >     >     >>> 1. If I emit this data using event time I have the problem
> described
> >     >
> >     >     >> where
> >     >
> >     >     >>> a geographical region with no new clicks or impressions
> will fail to
> >     >
> >     >     >> output
> >     >
> >     >     >>> results.
> >     >
> >     >     >>> 2. If I emit this data using system time I have the problem
> that when
> >     >
> >     >     >>> reprocessing data my window may not be ten minutes but 10
> hours if my
> >     >
> >     >     >>> processing is very fast so it dramatically changes the
> output.
> >     >
> >     >     >>>
> >     >
> >     >     >>> Maybe a hybrid solution works: I window by event time but
> trigger results
> >     >
> >     >     >>> by system time for windows that have updated? Not really
> sure the details
> >     >
> >     >     >>> of making that work. Does that work? Are there concrete
> examples where
> >     >
> >     >     >> you
> >     >
> >     >     >>> actually want the current behavior?
> >     >
> >     >     >>>
> >     >
> >     >     >>> -Jay
> >     >
> >     >     >>>
> >     >
> >     >     >>>
> >     >
> >     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
> arunmathew88@gmail.com> <ar...@gmail.com>
> >     >
> >     >     >>> wrote:
> >     >
> >     >     >>>
> >     >
> >     >     >>>> Hi All,
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> Thanks for the KIP. We were also in need of a mechanism to
> trigger
> >     >
> >     >     >>>> punctuate in the absence of events.
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> As I described in [
> >     >
> >     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
> >     >
> >     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
> >     >
> >     >     >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-
> 15926036
> >     >
> >     >     >>>> ],
> >     >
> >     >     >>>>
> >     >
> >     >     >>>>    - Our approached involved using the event time by
> default.
> >     >
> >     >     >>>>    - The method to check if there is any punctuate ready
> in the
> >     >
> >     >     >>>>    PunctuationQueue is triggered via the any event
> received by the
> >     >
> >     >     >> stream
> >     >
> >     >     >>>>    tread, or at the polling intervals in the absence of
> any events.
> >     >
> >     >     >>>>    - When we create Punctuate objects (which contains the
> next event
> >     >
> >     >     >> time
> >     >
> >     >     >>>>    for punctuation and interval), we also record the
> creation time
> >     >
> >     >     >>> (system
> >     >
> >     >     >>>>    time).
> >     >
> >     >     >>>>    - While checking for maturity of Punctuate Schedule by
> >     >
> >     >     >> mayBePunctuate
> >     >
> >     >     >>>>    method, we also check if the system clock has elapsed
> the punctuate
> >     >
> >     >     >>>>    interval since the schedule creation time.
> >     >
> >     >     >>>>    - In the absence of any event, or in the absence of any
> event for
> >     >
> >     >     >> one
> >     >
> >     >     >>>>    topic in the partition group assigned to the stream
> task, the system
> >     >
> >     >     >>>> time
> >     >
> >     >     >>>>    will elapse the interval and we trigger a punctuate
> using the
> >     >
> >     >     >> expected
> >     >
> >     >     >>>>    punctuation event time.
> >     >
> >     >     >>>>    - we then create the next punctuation schedule as
> punctuation event
> >     >
> >     >     >>> time
> >     >
> >     >     >>>>    + punctuation interval, [again recording the system
> time of creation
> >     >
> >     >     >>> of
> >     >
> >     >     >>>> the
> >     >
> >     >     >>>>    schedule].
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> We call this a Hybrid Punctuate. Of course, this approach
> has pros and
> >     >
> >     >     >>>> cons.
> >     >
> >     >     >>>> Pros
> >     >
> >     >     >>>>
> >     >
> >     >     >>>>    - Punctuates will happen in <punctuate interval> time
> duration at
> >     >
> >     >     >> max
> >     >
> >     >     >>> in
> >     >
> >     >     >>>>    terms of system time.
> >     >
> >     >     >>>>    - The semantics as a whole continues to revolve around
> event time.
> >     >
> >     >     >>>>    - We can use the old data [old timestamps] to rerun any
> experiments
> >     >
> >     >     >> or
> >     >
> >     >     >>>>    tests.
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> Cons
> >     >
> >     >     >>>>
> >     >
> >     >     >>>>    - In case the  <punctuate interval> is not a time
> duration [say
> >     >
> >     >     >>> logical
> >     >
> >     >     >>>>    time/event count], then the approach might not be
> meaningful.
> >     >
> >     >     >>>>    - In case there is a case where we have to wait for an
> actual event
> >     >
> >     >     >>> from
> >     >
> >     >     >>>>    a low event rate partition in the partition group, this
> approach
> >     >
> >     >     >> will
> >     >
> >     >     >>>> jump
> >     >
> >     >     >>>>    the gun.
> >     >
> >     >     >>>>    - in case the event processing cannot catch up with the
> event rate
> >     >
> >     >     >> and
> >     >
> >     >     >>>>    the expected timestamp events gets queued for long
> time, this
> >     >
> >     >     >> approach
> >     >
> >     >     >>>>    might jump the gun.
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> I believe the above approach and discussion goes close to
> the approach
> >     >
> >     >     >> A.
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> -----------
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> I like the idea of having an even count based punctuate.
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> -----------
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> I agree with the discussion around approach C, that we
> should provide
> >     >
> >     >     >> the
> >     >
> >     >     >>>> user with the option to choose system time or event time
> based
> >     >
> >     >     >>> punctuates.
> >     >
> >     >     >>>> But I believe that the user predominantly wants to use
> event time while
> >     >
> >     >     >>> not
> >     >
> >     >     >>>> missing out on regular punctuates due to event delays or
> event
> >     >
> >     >     >> absences.
> >     >
> >     >     >>>> Hence a complex punctuate option as Matthias mentioned
> (quoted below)
> >     >
> >     >     >>> would
> >     >
> >     >     >>>> be most apt.
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> "- We might want to add "complex" schedules later on
> (like, punctuate
> >     >
> >     >     >> on
> >     >
> >     >     >>>> every 10 seconds event-time or 60 seconds system-time
> whatever comes
> >     >
> >     >     >>>> first)."
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> -----------
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> I think I read somewhere that Kafka Streams started with
> System Time as
> >     >
> >     >     >>> the
> >     >
> >     >     >>>> punctuation standard, but was later changed to Event Time.
> I guess
> >     >
> >     >     >> there
> >     >
> >     >     >>>> would be some good reason behind it. As Kafka Streams want
> to evolve
> >     >
> >     >     >> more
> >     >
> >     >     >>>> on the Stream Processing front, I believe the emphasis on
> event time
> >     >
> >     >     >>> would
> >     >
> >     >     >>>> remain quite strong.
> >     >
> >     >     >>>>
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> With Regards,
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> Arun Mathew
> >     >
> >     >     >>>> Yahoo! JAPAN Corporation, Tokyo
> >     >
> >     >     >>>>
> >     >
> >     >     >>>>
> >     >
> >     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
> tobecker@tivo.com> <to...@tivo.com>
> >     >
> >     >     >> wrote:
> >     >
> >     >     >>>>
> >     >
> >     >     >>>>> Yeah I like PuncutationType much better; I just threw
> Time out there
> >     >
> >     >     >>>>> more as a strawman than an actual suggestion ;) I still
> think it's
> >     >
> >     >     >>>>> worth considering what this buys us over an additional
> callback. I
> >     >
> >     >     >>>>> foresee a number of punctuate implementations following
> this pattern:
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>> public void punctuate(PunctuationType type) {
> >     >
> >     >     >>>>>     switch (type) {
> >     >
> >     >     >>>>>         case EVENT_TIME:
> >     >
> >     >     >>>>>             methodA();
> >     >
> >     >     >>>>>             break;
> >     >
> >     >     >>>>>         case SYSTEM_TIME:
> >     >
> >     >     >>>>>             methodB();
> >     >
> >     >     >>>>>             break;
> >     >
> >     >     >>>>>     }
> >     >
> >     >     >>>>> }
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>> I guess one advantage of this approach is we could add
> additional
> >     >
> >     >     >>>>> punctuation types later in a backwards compatible way
> (like event
> >     >
> >     >     >> count
> >     >
> >     >     >>>>> as you mentioned).
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>> -Tommy
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
> >     >
> >     >     >>>>>> That sounds promising.
> >     >
> >     >     >>>>>>
> >     >
> >     >     >>>>>> I am just wondering if `Time` is the best name. Maybe we
> want to
> >     >
> >     >     >> add
> >     >
> >     >     >>>>>> other non-time based punctuations at some point later. I
> would
> >     >
> >     >     >>>>>> suggest
> >     >
> >     >     >>>>>>
> >     >
> >     >     >>>>>> enum PunctuationType {
> >     >
> >     >     >>>>>>   EVENT_TIME,
> >     >
> >     >     >>>>>>   SYSTEM_TIME,
> >     >
> >     >     >>>>>> }
> >     >
> >     >     >>>>>>
> >     >
> >     >     >>>>>> or similar. Just to keep the door open -- it's easier to
> add new
> >     >
> >     >     >>>>>> stuff
> >     >
> >     >     >>>>>> if the name is more generic.
> >     >
> >     >     >>>>>>
> >     >
> >     >     >>>>>>
> >     >
> >     >     >>>>>> -Matthias
> >     >
> >     >     >>>>>>
> >     >
> >     >     >>>>>>
> >     >
> >     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>> I agree that the framework providing and managing the
> notion of
> >     >
> >     >     >>>>>>> stream
> >     >
> >     >     >>>>>>> time is valuable and not something we would want to
> delegate to
> >     >
> >     >     >> the
> >     >
> >     >     >>>>>>> tasks. I'm not entirely convinced that a separate
> callback
> >     >
> >     >     >> (option
> >     >
> >     >     >>>>>>> C)
> >     >
> >     >     >>>>>>> is that messy (it could just be a default method with
> an empty
> >     >
> >     >     >>>>>>> implementation), but if we wanted a single API to
> handle both
> >     >
> >     >     >>>>>>> cases,
> >     >
> >     >     >>>>>>> how about something like the following?
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>> enum Time {
> >     >
> >     >     >>>>>>>    STREAM,
> >     >
> >     >     >>>>>>>    CLOCK
> >     >
> >     >     >>>>>>> }
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>> Then on ProcessorContext:
> >     >
> >     >     >>>>>>> context.schedule(Time time, long interval)  // We could
> allow
> >     >
> >     >     >> this
> >     >
> >     >     >>>>>>> to
> >     >
> >     >     >>>>>>> be called once for each value of time to mix
> approaches.
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>> Then the Processor API becomes:
> >     >
> >     >     >>>>>>> punctuate(Time time) // time here denotes which
> schedule resulted
> >     >
> >     >     >>>>>>> in
> >     >
> >     >     >>>>>>> this call.
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>> Thoughts?
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax
> wrote:
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> Thanks a lot for the KIP Michal,
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> I was thinking about the four options you proposed in
> more
> >     >
> >     >     >>>>>>>> details
> >     >
> >     >     >>>>>>>> and
> >     >
> >     >     >>>>>>>> this are my thoughts:
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> (A) You argue, that users can still "punctuate" on
> event-time
> >     >
> >     >     >> via
> >     >
> >     >     >>>>>>>> process(), but I am not sure if this is possible.
> Note, that
> >     >
> >     >     >>>>>>>> users
> >     >
> >     >     >>>>>>>> only
> >     >
> >     >     >>>>>>>> get record timestamps via context.timestamp(). Thus,
> users
> >     >
> >     >     >> would
> >     >
> >     >     >>>>>>>> need
> >     >
> >     >     >>>>>>>> to
> >     >
> >     >     >>>>>>>> track the time progress per partition (based on the
> partitions
> >     >
> >     >     >>>>>>>> they
> >     >
> >     >     >>>>>>>> obverse via context.partition(). (This alone puts a
> huge burden
> >     >
> >     >     >>>>>>>> on
> >     >
> >     >     >>>>>>>> the
> >     >
> >     >     >>>>>>>> user by itself.) However, users are not notified at
> startup
> >     >
> >     >     >> what
> >     >
> >     >     >>>>>>>> partitions are assigned, and user are not notified
> when
> >     >
> >     >     >>>>>>>> partitions
> >     >
> >     >     >>>>>>>> get
> >     >
> >     >     >>>>>>>> revoked. Because this information is not available,
> it's not
> >     >
> >     >     >>>>>>>> possible
> >     >
> >     >     >>>>>>>> to
> >     >
> >     >     >>>>>>>> "manually advance" stream-time, and thus event-time
> punctuation
> >     >
> >     >     >>>>>>>> within
> >     >
> >     >     >>>>>>>> process() seems not to be possible -- or do you see a
> way to
> >     >
> >     >     >> get
> >     >
> >     >     >>>>>>>> it
> >     >
> >     >     >>>>>>>> done? And even if, it might still be too clumsy to
> use.
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> (B) This does not allow to mix both approaches, thus
> limiting
> >     >
> >     >     >>>>>>>> what
> >     >
> >     >     >>>>>>>> users
> >     >
> >     >     >>>>>>>> can do.
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> (C) This should give all flexibility we need. However,
> just
> >     >
> >     >     >>>>>>>> adding
> >     >
> >     >     >>>>>>>> one
> >     >
> >     >     >>>>>>>> more method seems to be a solution that is too simple
> (cf my
> >     >
> >     >     >>>>>>>> comments
> >     >
> >     >     >>>>>>>> below).
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> (D) This might be hard to use. Also, I am not sure how
> a user
> >     >
> >     >     >>>>>>>> could
> >     >
> >     >     >>>>>>>> enable system-time and event-time punctuation in
> parallel.
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> Overall options (C) seems to be the most promising
> approach to
> >     >
> >     >     >>>>>>>> me.
> >     >
> >     >     >>>>>>>> Because I also favor a clean API, we might keep
> current
> >     >
> >     >     >>>>>>>> punctuate()
> >     >
> >     >     >>>>>>>> as-is, but deprecate it -- so we can remove it at some
> later
> >     >
> >     >     >>>>>>>> point
> >     >
> >     >     >>>>>>>> when
> >     >
> >     >     >>>>>>>> people use the "new punctuate API".
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> Couple of follow up questions:
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> - I am wondering, if we should have two callback
> methods or
> >     >
> >     >     >> just
> >     >
> >     >     >>>>>>>> one
> >     >
> >     >     >>>>>>>> (ie, a unified for system and event time punctuation
> or one for
> >     >
> >     >     >>>>>>>> each?).
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> - If we have one, how can the user figure out, which
> condition
> >     >
> >     >     >>>>>>>> did
> >     >
> >     >     >>>>>>>> trigger?
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> - How would the API look like, for registering
> different
> >     >
> >     >     >>>>>>>> punctuate
> >     >
> >     >     >>>>>>>> schedules? The "type" must be somehow defined?
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> - We might want to add "complex" schedules later on
> (like,
> >     >
> >     >     >>>>>>>> punctuate
> >     >
> >     >     >>>>>>>> on
> >     >
> >     >     >>>>>>>> every 10 seconds event-time or 60 seconds system-time
> whatever
> >     >
> >     >     >>>>>>>> comes
> >     >
> >     >     >>>>>>>> first). I don't say we should add this right away, but
> we might
> >     >
> >     >     >>>>>>>> want
> >     >
> >     >     >>>>>>>> to
> >     >
> >     >     >>>>>>>> define the API in a way, that it allows extensions
> like this
> >     >
> >     >     >>>>>>>> later
> >     >
> >     >     >>>>>>>> on,
> >     >
> >     >     >>>>>>>> without redesigning the API (ie, the API should be
> designed
> >     >
> >     >     >>>>>>>> extensible)
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> - Did you ever consider count-based punctuation?
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> I understand, that you would like to solve a simple
> problem,
> >     >
> >     >     >> but
> >     >
> >     >     >>>>>>>> we
> >     >
> >     >     >>>>>>>> learned from the past, that just "adding some API"
> quickly
> >     >
> >     >     >> leads
> >     >
> >     >     >>>>>>>> to a
> >     >
> >     >     >>>>>>>> not very well defined API that needs time consuming
> clean up
> >     >
> >     >     >>>>>>>> later on
> >     >
> >     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
> >     >
> >     >     >>>>>>>> punctuation
> >     >
> >     >     >>>>>>>> KIP
> >     >
> >     >     >>>>>>>> with this from the beginning on to avoid later painful
> >     >
> >     >     >> redesign.
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> -Matthias
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>>
> >     >
> >     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>> Thanks Thomas,
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>> I'm also wary of changing the existing semantics of
> >     >
> >     >     >> punctuate,
> >     >
> >     >     >>>>>>>>> for
> >     >
> >     >     >>>>>>>>> backward compatibility reasons, although I like the
> >     >
> >     >     >> conceptual
> >     >
> >     >     >>>>>>>>> simplicity of that option.
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>> Adding a new method to me feels safer but, in a way,
> uglier.
> >     >
> >     >     >> I
> >     >
> >     >     >>>>>>>>> added
> >     >
> >     >     >>>>>>>>> this to the KIP now as option (C).
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>> The TimestampExtractor mechanism is actually more
> flexible,
> >     >
> >     >     >> as
> >     >
> >     >     >>>>>>>>> it
> >     >
> >     >     >>>>>>>>> allows
> >     >
> >     >     >>>>>>>>> you to return any value, you're not limited to event
> time or
> >     >
> >     >     >>>>>>>>> system
> >     >
> >     >     >>>>>>>>> time
> >     >
> >     >     >>>>>>>>> (although I don't see an actual use case where you
> might need
> >     >
> >     >     >>>>>>>>> anything
> >     >
> >     >     >>>>>>>>> else then those two). Hence I also proposed the
> option to
> >     >
> >     >     >> allow
> >     >
> >     >     >>>>>>>>> users
> >     >
> >     >     >>>>>>>>> to, effectively, decide what "stream time" is for
> them given
> >     >
> >     >     >>>>>>>>> the
> >     >
> >     >     >>>>>>>>> presence or absence of messages, much like they can
> decide
> >     >
> >     >     >> what
> >     >
> >     >     >>>>>>>>> msg
> >     >
> >     >     >>>>>>>>> time
> >     >
> >     >     >>>>>>>>> means for them using the TimestampExtractor. What do
> you
> >     >
> >     >     >> think
> >     >
> >     >     >>>>>>>>> about
> >     >
> >     >     >>>>>>>>> that? This is probably most flexible but also most
> >     >
> >     >     >> complicated.
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>> All comments appreciated.
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>> Cheers,
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>> Michal
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>>
> >     >
> >     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
> >     >
> >     >     >>>>>>>>>>
> >     >
> >     >     >>>>>>>>>>
> >     >
> >     >     >>>>>>>>>> Although I fully agree we need a way to trigger
> periodic
> >     >
> >     >     >>>>>>>>>> processing
> >     >
> >     >     >>>>>>>>>> that is independent from whether and when messages
> arrive,
> >     >
> >     >     >>>>>>>>>> I'm
> >     >
> >     >     >>>>>>>>>> not sure
> >     >
> >     >     >>>>>>>>>> I like the idea of changing the existing semantics
> across
> >     >
> >     >     >> the
> >     >
> >     >     >>>>>>>>>> board.
> >     >
> >     >     >>>>>>>>>> What if we added an additional callback to Processor
> that
> >     >
> >     >     >> can
> >     >
> >     >     >>>>>>>>>> be
> >     >
> >     >     >>>>>>>>>> scheduled similarly to punctuate() but was always
> called at
> >     >
> >     >     >>>>>>>>>> fixed, wall
> >     >
> >     >     >>>>>>>>>> clock based intervals? This way you wouldn't have to
> give
> >     >
> >     >     >> up
> >     >
> >     >     >>>>>>>>>> the
> >     >
> >     >     >>>>>>>>>> notion
> >     >
> >     >     >>>>>>>>>> of stream time to be able to do periodic processing.
> >     >
> >     >     >>>>>>>>>>
> >     >
> >     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki
> wrote:
> >     >
> >     >     >>>>>>>>>>>
> >     >
> >     >     >>>>>>>>>>>
> >     >
> >     >     >>>>>>>>>>> Hi all,
> >     >
> >     >     >>>>>>>>>>>
> >     >
> >     >     >>>>>>>>>>> I have created a draft for KIP-138: Change
> punctuate
> >     >
> >     >     >>>>>>>>>>> semantics
> >     >
> >     >     >>>>>>>>>>> <https://cwiki.apache.org/
> confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
> confluence/display/KAFKA/KIP->
> >     >
> >     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
> 138%
> >     >
> >     >     >>>>>>>>>>> 3A+C
> >     >
> >     >     >>>>>>>>>>> hange+
> >     >
> >     >     >>>>>>>>>>> punctuate+semantics>
> >     >
> >     >     >>>>>>>>>>> .
> >     >
> >     >     >>>>>>>>>>>
> >     >
> >     >     >>>>>>>>>>> Appreciating there can be different views on
> system-time
> >     >
> >     >     >> vs
> >     >
> >     >     >>>>>>>>>>> event-
> >     >
> >     >     >>>>>>>>>>> time
> >     >
> >     >     >>>>>>>>>>> semantics for punctuation depending on use-case and
> the
> >     >
> >     >     >>>>>>>>>>> importance of
> >     >
> >     >     >>>>>>>>>>> backwards compatibility of any such change, I've
> left it
> >     >
> >     >     >>>>>>>>>>> quite
> >     >
> >     >     >>>>>>>>>>> open
> >     >
> >     >     >>>>>>>>>>> and
> >     >
> >     >     >>>>>>>>>>> hope to fill in more info as the discussion
> progresses.
> >     >
> >     >     >>>>>>>>>>>
> >     >
> >     >     >>>>>>>>>>> Thanks,
> >     >
> >     >     >>>>>>>>>>> Michal
> >     >
> >     >     >>>>>>> --
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>>     Tommy Becker
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>>     Senior Software Engineer
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>>     tivo.com
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>> ________________________________
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>>>> This email and any attachments may contain confidential
> and
> >     >
> >     >     >>>>>>> privileged material for the sole use of the intended
> recipient.
> >     >
> >     >     >> Any
> >     >
> >     >     >>>>>>> review, copying, or distribution of this email (or any
> >     >
> >     >     >> attachments)
> >     >
> >     >     >>>>>>> by others is prohibited. If you are not the intended
> recipient,
> >     >
> >     >     >>>>>>> please contact the sender immediately and permanently
> delete this
> >     >
> >     >     >>>>>>> email and any attachments. No employee or agent of TiVo
> Inc. is
> >     >
> >     >     >>>>>>> authorized to conclude any binding agreement on behalf
> of TiVo
> >     >
> >     >     >> Inc.
> >     >
> >     >     >>>>>>> by email. Binding agreements with TiVo Inc. may only be
> made by a
> >     >
> >     >     >>>>>>> signed written agreement.
> >     >
> >     >     >>>>>>>
> >     >
> >     >     >>>>> --
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>>     Tommy Becker
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>>     Senior Software Engineer
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>>     tivo.com
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>> ________________________________
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>> This email and any attachments may contain confidential
> and
> >     >
> >     >     >> privileged
> >     >
> >     >     >>>>> material for the sole use of the intended recipient. Any
> review,
> >     >
> >     >     >>> copying,
> >     >
> >     >     >>>>> or distribution of this email (or any attachments) by
> others is
> >     >
> >     >     >>>> prohibited.
> >     >
> >     >     >>>>> If you are not the intended recipient, please contact the
> sender
> >     >
> >     >     >>>>> immediately and permanently delete this email and any
> attachments. No
> >     >
> >     >     >>>>> employee or agent of TiVo Inc. is authorized to conclude
> any binding
> >     >
> >     >     >>>>> agreement on behalf of TiVo Inc. by email. Binding
> agreements with
> >     >
> >     >     >> TiVo
> >     >
> >     >     >>>>> Inc. may only be made by a signed written agreement.
> >     >
> >     >     >>>>>
> >     >
> >     >     >>>>
> >     >
> >     >     >>>
> >     >
> >     >     >>
> >     >
> >     >     >
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >
> >     >
> >     > --
> >     >
> >     > <http://www.openbet.com/>
> >     >
> >     > *Michal Borowiecki*
> >     >
> >     > *Senior Software Engineer L4*
> >     >
> >     > *T: *
> >     >
> >     > +44 208 742 1600 <+44%2020%208742%201600>
> >     >
> >     > +44 203 249 8448 <+44%2020%203249%208448>
> >     >
> >     >
> >     >
> >     > *E: *
> >     >
> >     > michal.borowiecki@openbet.com
> >     >
> >     > *W: *
> >     >
> >     > www.openbet.com
> >     >
> >     > *OpenBet Ltd*
> >     >
> >     > Chiswick Park Building 9
> >     >
> >     > 566 Chiswick High Rd
> >     >
> >     > London
> >     >
> >     > W4 5XT
> >     >
> >     > UK
> >     >
> >     > <https://www.openbet.com/email_promo>
> >     >
> >     >
> >     >
> >     > This message is confidential and intended only for the addressee.
> If you
> >     > have received this message in error, please immediately notify the
> >     > postmaster@openbet.com and delete it from your system as well as
> any
> >     > copies. The content of e-mails as well as traffic data may be
> monitored by
> >     > OpenBet for employment and security purposes. To protect the
> environment
> >     > please do not print this e-mail unless necessary. OpenBet Ltd.
> Registered
> >     > Office: Chiswick Park Building 9, 566 Chiswick High Road, London,
> W4 5XT,
> >     > United Kingdom. A company registered in England and Wales.
> Registered no.
> >     > 3134634. VAT no. GB927523612
> >     >
> >     >
> >     >
> >
> >
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Arun Mathew <ar...@gmail.com>.
Thanks Matthias.
Sure, will correct it right away.

On 11-Apr-2017 8:07 AM, "Matthias J. Sax" <ma...@confluent.io> wrote:

Thanks for preparing this page!

About terminology:

You introduce the term "event time" -- but we should call this "stream
time" -- "stream time" is whatever TimestampExtractor returns and this
could be event time, ingestion time, or processing/wall-clock time.

Does this make sense to you?



-Matthias


On 4/10/17 4:58 AM, Arun Mathew wrote:
> Thanks Ewen.
>
> @Michal, @all, I have created a child page to start the Use Cases
discussion [https://cwiki.apache.org/confluence/display/KAFKA/
Punctuate+Use+Cases]. Please go through it and give your comments.
>
> @Tianji, Sorry for the delay. I am trying to make the patch public.
>
> --
> Arun Mathew
>
> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> wrote:
>
>     Arun,
>
>     I've given you permission to edit the wiki. Let me know if you run
into any
>     issues.
>
>     -Ewen
>
>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <am...@yahoo-corp.jp>
wrote:
>
>     > Thanks Michal. I don’t have the access yet [arunmathew88]. Should I
be
>     > sending a separate mail for this?
>     >
>     > I thought one of the person following this thread would be able to
give me
>     > access.
>     >
>     >
>     >
>     > *From: *Michal Borowiecki <mi...@openbet.com>
>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>     > *Date: *Friday, April 7, 2017 at 17:16
>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
>     >
>     >
>     >
>     > Hi Arun,
>     >
>     > I was thinking along the same lines as you, listing the use cases
on the
>     > wiki, but didn't find time to get around doing that yet.
>     > Don't mind if you do it if you have access now.
>     > I was thinking it would be nice if, once we have the use cases
listed,
>     > people could use likes to up-vote the use cases similar to what
they're
>     > working on.
>     >
>     > I should have a bit more time to action this in the next few days,
but
>     > happy for you to do it if you can beat me to it ;-)
>     >
>     > Cheers,
>     > Michal
>     >
>     > On 07/04/17 04:39, Arun Mathew wrote:
>     >
>     > Sure, Thanks Matthias. My id is [arunmathew88].
>     >
>     >
>     >
>     > Of course. I was thinking of a subpage where people can collaborate.
>     >
>     >
>     >
>     > Will do as per Michael’s suggestion.
>     >
>     >
>     >
>     > Regards,
>     >
>     > Arun Mathew
>     >
>     >
>     >
>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <
matthias@confluent.io> wrote:
>     >
>     >
>     >
>     >     Please share your Wiki-ID and a committer can give you write
access.
>     >
>     >
>     >
>     >     Btw: as you did not initiate the KIP, you should not change the
KIP
>     >
>     >     without the permission of the original author -- in this case
Michael.
>     >
>     >
>     >
>     >     So you might also just share your thought over the mailing list
and
>     >
>     >     Michael can update the KIP page. Or, as an alternative, just
create a
>     >
>     >     subpage for the KIP page.
>     >
>     >
>     >
>     >     @Michael: WDYT?
>     >
>     >
>     >
>     >
>     >
>     >     -Matthias
>     >
>     >
>     >
>     >
>     >
>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>     >
>     >     > Hi Jay,
>     >
>     >     >           Thanks for the advise, I would like to list down
the use cases as
>     >
>     >     > per your suggestion. But it seems I don't have write
permission to the
>     >
>     >     > Apache Kafka Confluent Space. Whom shall I request for it?
>     >
>     >     >
>     >
>     >     > Regarding your last question. We are using a patch in our
production system
>     >
>     >     > which does exactly this.
>     >
>     >     > We window by the event time, but trigger punctuate in
<punctuate interval>
>     >
>     >     > duration of system time, in the absence of an event crossing
the punctuate
>     >
>     >     > event time.
>     >
>     >     >
>     >
>     >     > We are using Kafka Stream for our Audit Trail, where we need
to output the
>     >
>     >     > event counts on each topic on each cluster aggregated over a
1 minute
>     >
>     >     > window. We have to use event time to be able to cross check
the counts. But
>     >
>     >     > we need to trigger punctuate [aggregate event pushes] by
system time in the
>     >
>     >     > absence of events. Otherwise the event counts for unexpired
windows would
>     >
>     >     > be 0 which is bad.
>     >
>     >     >
>     >
>     >     > "Maybe a hybrid solution works: I window by event time but
trigger results
>     >
>     >     > by system time for windows that have updated? Not really sure
the details
>     >
>     >     > of making that work. Does that work? Are there concrete
examples where you
>     >
>     >     > actually want the current behavior?"
>     >
>     >     >
>     >
>     >     > --
>     >
>     >     > With Regards,
>     >
>     >     >
>     >
>     >     > Arun Mathew
>     >
>     >     > Yahoo! JAPAN Corporation
>     >
>     >     >
>     >
>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com>
<sk...@gmail.com> wrote:
>     >
>     >     >
>     >
>     >     >> Hi Jay,
>     >
>     >     >>
>     >
>     >     >> The hybrid solution is exactly what I expect and need for
our use cases
>     >
>     >     >> when dealing with telecom data.
>     >
>     >     >>
>     >
>     >     >> Thanks
>     >
>     >     >> Tianji
>     >
>     >     >>
>     >
>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io>
<ja...@confluent.io> wrote:
>     >
>     >     >>
>     >
>     >     >>> Hey guys,
>     >
>     >     >>>
>     >
>     >     >>> One thing I've always found super important for this kind
of design work
>     >
>     >     >> is
>     >
>     >     >>> to do a really good job of cataloging the landscape of use
cases and how
>     >
>     >     >>> prevalent each one is. By that I mean not just listing lots
of uses, but
>     >
>     >     >>> also grouping them into categories that functionally need
the same thing.
>     >
>     >     >>> In the absence of this it is very hard to reason about
design proposals.
>     >
>     >     >>> From the proposals so far I think we have a lot of
discussion around
>     >
>     >     >>> possible apis, but less around what the user needs for
different use
>     >
>     >     >> cases
>     >
>     >     >>> and how they would implement that using the api.
>     >
>     >     >>>
>     >
>     >     >>> Here is an example:
>     >
>     >     >>> You aggregate click and impression data for a reddit like
site. Every ten
>     >
>     >     >>> minutes you want to output a ranked list of the top 10
articles ranked by
>     >
>     >     >>> clicks/impressions for each geographical area. I want to be
able run this
>     >
>     >     >>> in steady state as well as rerun to regenerate results (or
catch up if it
>     >
>     >     >>> crashes).
>     >
>     >     >>>
>     >
>     >     >>> There are a couple of tricky things that seem to make this
hard with
>     >
>     >     >> either
>     >
>     >     >>> of the options proposed:
>     >
>     >     >>> 1. If I emit this data using event time I have the problem
described
>     >
>     >     >> where
>     >
>     >     >>> a geographical region with no new clicks or impressions
will fail to
>     >
>     >     >> output
>     >
>     >     >>> results.
>     >
>     >     >>> 2. If I emit this data using system time I have the problem
that when
>     >
>     >     >>> reprocessing data my window may not be ten minutes but 10
hours if my
>     >
>     >     >>> processing is very fast so it dramatically changes the
output.
>     >
>     >     >>>
>     >
>     >     >>> Maybe a hybrid solution works: I window by event time but
trigger results
>     >
>     >     >>> by system time for windows that have updated? Not really
sure the details
>     >
>     >     >>> of making that work. Does that work? Are there concrete
examples where
>     >
>     >     >> you
>     >
>     >     >>> actually want the current behavior?
>     >
>     >     >>>
>     >
>     >     >>> -Jay
>     >
>     >     >>>
>     >
>     >     >>>
>     >
>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <
arunmathew88@gmail.com> <ar...@gmail.com>
>     >
>     >     >>> wrote:
>     >
>     >     >>>
>     >
>     >     >>>> Hi All,
>     >
>     >     >>>>
>     >
>     >     >>>> Thanks for the KIP. We were also in need of a mechanism to
trigger
>     >
>     >     >>>> punctuate in the absence of events.
>     >
>     >     >>>>
>     >
>     >     >>>> As I described in [
>     >
>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>     >
>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>     >
>     >     >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-
15926036
>     >
>     >     >>>> ],
>     >
>     >     >>>>
>     >
>     >     >>>>    - Our approached involved using the event time by
default.
>     >
>     >     >>>>    - The method to check if there is any punctuate ready
in the
>     >
>     >     >>>>    PunctuationQueue is triggered via the any event
received by the
>     >
>     >     >> stream
>     >
>     >     >>>>    tread, or at the polling intervals in the absence of
any events.
>     >
>     >     >>>>    - When we create Punctuate objects (which contains the
next event
>     >
>     >     >> time
>     >
>     >     >>>>    for punctuation and interval), we also record the
creation time
>     >
>     >     >>> (system
>     >
>     >     >>>>    time).
>     >
>     >     >>>>    - While checking for maturity of Punctuate Schedule by
>     >
>     >     >> mayBePunctuate
>     >
>     >     >>>>    method, we also check if the system clock has elapsed
the punctuate
>     >
>     >     >>>>    interval since the schedule creation time.
>     >
>     >     >>>>    - In the absence of any event, or in the absence of any
event for
>     >
>     >     >> one
>     >
>     >     >>>>    topic in the partition group assigned to the stream
task, the system
>     >
>     >     >>>> time
>     >
>     >     >>>>    will elapse the interval and we trigger a punctuate
using the
>     >
>     >     >> expected
>     >
>     >     >>>>    punctuation event time.
>     >
>     >     >>>>    - we then create the next punctuation schedule as
punctuation event
>     >
>     >     >>> time
>     >
>     >     >>>>    + punctuation interval, [again recording the system
time of creation
>     >
>     >     >>> of
>     >
>     >     >>>> the
>     >
>     >     >>>>    schedule].
>     >
>     >     >>>>
>     >
>     >     >>>> We call this a Hybrid Punctuate. Of course, this approach
has pros and
>     >
>     >     >>>> cons.
>     >
>     >     >>>> Pros
>     >
>     >     >>>>
>     >
>     >     >>>>    - Punctuates will happen in <punctuate interval> time
duration at
>     >
>     >     >> max
>     >
>     >     >>> in
>     >
>     >     >>>>    terms of system time.
>     >
>     >     >>>>    - The semantics as a whole continues to revolve around
event time.
>     >
>     >     >>>>    - We can use the old data [old timestamps] to rerun any
experiments
>     >
>     >     >> or
>     >
>     >     >>>>    tests.
>     >
>     >     >>>>
>     >
>     >     >>>> Cons
>     >
>     >     >>>>
>     >
>     >     >>>>    - In case the  <punctuate interval> is not a time
duration [say
>     >
>     >     >>> logical
>     >
>     >     >>>>    time/event count], then the approach might not be
meaningful.
>     >
>     >     >>>>    - In case there is a case where we have to wait for an
actual event
>     >
>     >     >>> from
>     >
>     >     >>>>    a low event rate partition in the partition group, this
approach
>     >
>     >     >> will
>     >
>     >     >>>> jump
>     >
>     >     >>>>    the gun.
>     >
>     >     >>>>    - in case the event processing cannot catch up with the
event rate
>     >
>     >     >> and
>     >
>     >     >>>>    the expected timestamp events gets queued for long
time, this
>     >
>     >     >> approach
>     >
>     >     >>>>    might jump the gun.
>     >
>     >     >>>>
>     >
>     >     >>>> I believe the above approach and discussion goes close to
the approach
>     >
>     >     >> A.
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I like the idea of having an even count based punctuate.
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I agree with the discussion around approach C, that we
should provide
>     >
>     >     >> the
>     >
>     >     >>>> user with the option to choose system time or event time
based
>     >
>     >     >>> punctuates.
>     >
>     >     >>>> But I believe that the user predominantly wants to use
event time while
>     >
>     >     >>> not
>     >
>     >     >>>> missing out on regular punctuates due to event delays or
event
>     >
>     >     >> absences.
>     >
>     >     >>>> Hence a complex punctuate option as Matthias mentioned
(quoted below)
>     >
>     >     >>> would
>     >
>     >     >>>> be most apt.
>     >
>     >     >>>>
>     >
>     >     >>>> "- We might want to add "complex" schedules later on
(like, punctuate
>     >
>     >     >> on
>     >
>     >     >>>> every 10 seconds event-time or 60 seconds system-time
whatever comes
>     >
>     >     >>>> first)."
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I think I read somewhere that Kafka Streams started with
System Time as
>     >
>     >     >>> the
>     >
>     >     >>>> punctuation standard, but was later changed to Event Time.
I guess
>     >
>     >     >> there
>     >
>     >     >>>> would be some good reason behind it. As Kafka Streams want
to evolve
>     >
>     >     >> more
>     >
>     >     >>>> on the Stream Processing front, I believe the emphasis on
event time
>     >
>     >     >>> would
>     >
>     >     >>>> remain quite strong.
>     >
>     >     >>>>
>     >
>     >     >>>>
>     >
>     >     >>>> With Regards,
>     >
>     >     >>>>
>     >
>     >     >>>> Arun Mathew
>     >
>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>     >
>     >     >>>>
>     >
>     >     >>>>
>     >
>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <
tobecker@tivo.com> <to...@tivo.com>
>     >
>     >     >> wrote:
>     >
>     >     >>>>
>     >
>     >     >>>>> Yeah I like PuncutationType much better; I just threw
Time out there
>     >
>     >     >>>>> more as a strawman than an actual suggestion ;) I still
think it's
>     >
>     >     >>>>> worth considering what this buys us over an additional
callback. I
>     >
>     >     >>>>> foresee a number of punctuate implementations following
this pattern:
>     >
>     >     >>>>>
>     >
>     >     >>>>> public void punctuate(PunctuationType type) {
>     >
>     >     >>>>>     switch (type) {
>     >
>     >     >>>>>         case EVENT_TIME:
>     >
>     >     >>>>>             methodA();
>     >
>     >     >>>>>             break;
>     >
>     >     >>>>>         case SYSTEM_TIME:
>     >
>     >     >>>>>             methodB();
>     >
>     >     >>>>>             break;
>     >
>     >     >>>>>     }
>     >
>     >     >>>>> }
>     >
>     >     >>>>>
>     >
>     >     >>>>> I guess one advantage of this approach is we could add
additional
>     >
>     >     >>>>> punctuation types later in a backwards compatible way
(like event
>     >
>     >     >> count
>     >
>     >     >>>>> as you mentioned).
>     >
>     >     >>>>>
>     >
>     >     >>>>> -Tommy
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
>     >
>     >     >>>>>> That sounds promising.
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> I am just wondering if `Time` is the best name. Maybe we
want to
>     >
>     >     >> add
>     >
>     >     >>>>>> other non-time based punctuations at some point later. I
would
>     >
>     >     >>>>>> suggest
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> enum PunctuationType {
>     >
>     >     >>>>>>   EVENT_TIME,
>     >
>     >     >>>>>>   SYSTEM_TIME,
>     >
>     >     >>>>>> }
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> or similar. Just to keep the door open -- it's easier to
add new
>     >
>     >     >>>>>> stuff
>     >
>     >     >>>>>> if the name is more generic.
>     >
>     >     >>>>>>
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> -Matthias
>     >
>     >     >>>>>>
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> I agree that the framework providing and managing the
notion of
>     >
>     >     >>>>>>> stream
>     >
>     >     >>>>>>> time is valuable and not something we would want to
delegate to
>     >
>     >     >> the
>     >
>     >     >>>>>>> tasks. I'm not entirely convinced that a separate
callback
>     >
>     >     >> (option
>     >
>     >     >>>>>>> C)
>     >
>     >     >>>>>>> is that messy (it could just be a default method with
an empty
>     >
>     >     >>>>>>> implementation), but if we wanted a single API to
handle both
>     >
>     >     >>>>>>> cases,
>     >
>     >     >>>>>>> how about something like the following?
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> enum Time {
>     >
>     >     >>>>>>>    STREAM,
>     >
>     >     >>>>>>>    CLOCK
>     >
>     >     >>>>>>> }
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Then on ProcessorContext:
>     >
>     >     >>>>>>> context.schedule(Time time, long interval)  // We could
allow
>     >
>     >     >> this
>     >
>     >     >>>>>>> to
>     >
>     >     >>>>>>> be called once for each value of time to mix approaches.
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Then the Processor API becomes:
>     >
>     >     >>>>>>> punctuate(Time time) // time here denotes which
schedule resulted
>     >
>     >     >>>>>>> in
>     >
>     >     >>>>>>> this call.
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Thoughts?
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax
wrote:
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> I was thinking about the four options you proposed in
more
>     >
>     >     >>>>>>>> details
>     >
>     >     >>>>>>>> and
>     >
>     >     >>>>>>>> this are my thoughts:
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (A) You argue, that users can still "punctuate" on
event-time
>     >
>     >     >> via
>     >
>     >     >>>>>>>> process(), but I am not sure if this is possible.
Note, that
>     >
>     >     >>>>>>>> users
>     >
>     >     >>>>>>>> only
>     >
>     >     >>>>>>>> get record timestamps via context.timestamp(). Thus,
users
>     >
>     >     >> would
>     >
>     >     >>>>>>>> need
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> track the time progress per partition (based on the
partitions
>     >
>     >     >>>>>>>> they
>     >
>     >     >>>>>>>> obverse via context.partition(). (This alone puts a
huge burden
>     >
>     >     >>>>>>>> on
>     >
>     >     >>>>>>>> the
>     >
>     >     >>>>>>>> user by itself.) However, users are not notified at
startup
>     >
>     >     >> what
>     >
>     >     >>>>>>>> partitions are assigned, and user are not notified when
>     >
>     >     >>>>>>>> partitions
>     >
>     >     >>>>>>>> get
>     >
>     >     >>>>>>>> revoked. Because this information is not available,
it's not
>     >
>     >     >>>>>>>> possible
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> "manually advance" stream-time, and thus event-time
punctuation
>     >
>     >     >>>>>>>> within
>     >
>     >     >>>>>>>> process() seems not to be possible -- or do you see a
way to
>     >
>     >     >> get
>     >
>     >     >>>>>>>> it
>     >
>     >     >>>>>>>> done? And even if, it might still be too clumsy to use.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (B) This does not allow to mix both approaches, thus
limiting
>     >
>     >     >>>>>>>> what
>     >
>     >     >>>>>>>> users
>     >
>     >     >>>>>>>> can do.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (C) This should give all flexibility we need. However,
just
>     >
>     >     >>>>>>>> adding
>     >
>     >     >>>>>>>> one
>     >
>     >     >>>>>>>> more method seems to be a solution that is too simple
(cf my
>     >
>     >     >>>>>>>> comments
>     >
>     >     >>>>>>>> below).
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (D) This might be hard to use. Also, I am not sure how
a user
>     >
>     >     >>>>>>>> could
>     >
>     >     >>>>>>>> enable system-time and event-time punctuation in
parallel.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Overall options (C) seems to be the most promising
approach to
>     >
>     >     >>>>>>>> me.
>     >
>     >     >>>>>>>> Because I also favor a clean API, we might keep current
>     >
>     >     >>>>>>>> punctuate()
>     >
>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it at some
later
>     >
>     >     >>>>>>>> point
>     >
>     >     >>>>>>>> when
>     >
>     >     >>>>>>>> people use the "new punctuate API".
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Couple of follow up questions:
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - I am wondering, if we should have two callback
methods or
>     >
>     >     >> just
>     >
>     >     >>>>>>>> one
>     >
>     >     >>>>>>>> (ie, a unified for system and event time punctuation
or one for
>     >
>     >     >>>>>>>> each?).
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - If we have one, how can the user figure out, which
condition
>     >
>     >     >>>>>>>> did
>     >
>     >     >>>>>>>> trigger?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - How would the API look like, for registering
different
>     >
>     >     >>>>>>>> punctuate
>     >
>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - We might want to add "complex" schedules later on
(like,
>     >
>     >     >>>>>>>> punctuate
>     >
>     >     >>>>>>>> on
>     >
>     >     >>>>>>>> every 10 seconds event-time or 60 seconds system-time
whatever
>     >
>     >     >>>>>>>> comes
>     >
>     >     >>>>>>>> first). I don't say we should add this right away, but
we might
>     >
>     >     >>>>>>>> want
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> define the API in a way, that it allows extensions
like this
>     >
>     >     >>>>>>>> later
>     >
>     >     >>>>>>>> on,
>     >
>     >     >>>>>>>> without redesigning the API (ie, the API should be
designed
>     >
>     >     >>>>>>>> extensible)
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - Did you ever consider count-based punctuation?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> I understand, that you would like to solve a simple
problem,
>     >
>     >     >> but
>     >
>     >     >>>>>>>> we
>     >
>     >     >>>>>>>> learned from the past, that just "adding some API"
quickly
>     >
>     >     >> leads
>     >
>     >     >>>>>>>> to a
>     >
>     >     >>>>>>>> not very well defined API that needs time consuming
clean up
>     >
>     >     >>>>>>>> later on
>     >
>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
>     >
>     >     >>>>>>>> punctuation
>     >
>     >     >>>>>>>> KIP
>     >
>     >     >>>>>>>> with this from the beginning on to avoid later painful
>     >
>     >     >> redesign.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> -Matthias
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Thanks Thomas,
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> I'm also wary of changing the existing semantics of
>     >
>     >     >> punctuate,
>     >
>     >     >>>>>>>>> for
>     >
>     >     >>>>>>>>> backward compatibility reasons, although I like the
>     >
>     >     >> conceptual
>     >
>     >     >>>>>>>>> simplicity of that option.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Adding a new method to me feels safer but, in a way,
uglier.
>     >
>     >     >> I
>     >
>     >     >>>>>>>>> added
>     >
>     >     >>>>>>>>> this to the KIP now as option (C).
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> The TimestampExtractor mechanism is actually more
flexible,
>     >
>     >     >> as
>     >
>     >     >>>>>>>>> it
>     >
>     >     >>>>>>>>> allows
>     >
>     >     >>>>>>>>> you to return any value, you're not limited to event
time or
>     >
>     >     >>>>>>>>> system
>     >
>     >     >>>>>>>>> time
>     >
>     >     >>>>>>>>> (although I don't see an actual use case where you
might need
>     >
>     >     >>>>>>>>> anything
>     >
>     >     >>>>>>>>> else then those two). Hence I also proposed the
option to
>     >
>     >     >> allow
>     >
>     >     >>>>>>>>> users
>     >
>     >     >>>>>>>>> to, effectively, decide what "stream time" is for
them given
>     >
>     >     >>>>>>>>> the
>     >
>     >     >>>>>>>>> presence or absence of messages, much like they can
decide
>     >
>     >     >> what
>     >
>     >     >>>>>>>>> msg
>     >
>     >     >>>>>>>>> time
>     >
>     >     >>>>>>>>> means for them using the TimestampExtractor. What do
you
>     >
>     >     >> think
>     >
>     >     >>>>>>>>> about
>     >
>     >     >>>>>>>>> that? This is probably most flexible but also most
>     >
>     >     >> complicated.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> All comments appreciated.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Cheers,
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Michal
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>> Although I fully agree we need a way to trigger
periodic
>     >
>     >     >>>>>>>>>> processing
>     >
>     >     >>>>>>>>>> that is independent from whether and when messages
arrive,
>     >
>     >     >>>>>>>>>> I'm
>     >
>     >     >>>>>>>>>> not sure
>     >
>     >     >>>>>>>>>> I like the idea of changing the existing semantics
across
>     >
>     >     >> the
>     >
>     >     >>>>>>>>>> board.
>     >
>     >     >>>>>>>>>> What if we added an additional callback to Processor
that
>     >
>     >     >> can
>     >
>     >     >>>>>>>>>> be
>     >
>     >     >>>>>>>>>> scheduled similarly to punctuate() but was always
called at
>     >
>     >     >>>>>>>>>> fixed, wall
>     >
>     >     >>>>>>>>>> clock based intervals? This way you wouldn't have to
give
>     >
>     >     >> up
>     >
>     >     >>>>>>>>>> the
>     >
>     >     >>>>>>>>>> notion
>     >
>     >     >>>>>>>>>> of stream time to be able to do periodic processing.
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki
wrote:
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Hi all,
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
>     >
>     >     >>>>>>>>>>> semantics
>     >
>     >     >>>>>>>>>>> <https://cwiki.apache.org/
confluence/display/KAFKA/KIP- <https://cwiki.apache.org/
confluence/display/KAFKA/KIP->
>     >
>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>>
138%
>     >
>     >     >>>>>>>>>>> 3A+C
>     >
>     >     >>>>>>>>>>> hange+
>     >
>     >     >>>>>>>>>>> punctuate+semantics>
>     >
>     >     >>>>>>>>>>> .
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Appreciating there can be different views on
system-time
>     >
>     >     >> vs
>     >
>     >     >>>>>>>>>>> event-
>     >
>     >     >>>>>>>>>>> time
>     >
>     >     >>>>>>>>>>> semantics for punctuation depending on use-case and
the
>     >
>     >     >>>>>>>>>>> importance of
>     >
>     >     >>>>>>>>>>> backwards compatibility of any such change, I've
left it
>     >
>     >     >>>>>>>>>>> quite
>     >
>     >     >>>>>>>>>>> open
>     >
>     >     >>>>>>>>>>> and
>     >
>     >     >>>>>>>>>>> hope to fill in more info as the discussion
progresses.
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Thanks,
>     >
>     >     >>>>>>>>>>> Michal
>     >
>     >     >>>>>>> --
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     Tommy Becker
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     Senior Software Engineer
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     tivo.com
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> ________________________________
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> This email and any attachments may contain confidential
and
>     >
>     >     >>>>>>> privileged material for the sole use of the intended
recipient.
>     >
>     >     >> Any
>     >
>     >     >>>>>>> review, copying, or distribution of this email (or any
>     >
>     >     >> attachments)
>     >
>     >     >>>>>>> by others is prohibited. If you are not the intended
recipient,
>     >
>     >     >>>>>>> please contact the sender immediately and permanently
delete this
>     >
>     >     >>>>>>> email and any attachments. No employee or agent of TiVo
Inc. is
>     >
>     >     >>>>>>> authorized to conclude any binding agreement on behalf
of TiVo
>     >
>     >     >> Inc.
>     >
>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may only be
made by a
>     >
>     >     >>>>>>> signed written agreement.
>     >
>     >     >>>>>>>
>     >
>     >     >>>>> --
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>>     Tommy Becker
>     >
>     >     >>>>>
>     >
>     >     >>>>>     Senior Software Engineer
>     >
>     >     >>>>>
>     >
>     >     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
>     >
>     >     >>>>>
>     >
>     >     >>>>>     tivo.com
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>> ________________________________
>     >
>     >     >>>>>
>     >
>     >     >>>>> This email and any attachments may contain confidential
and
>     >
>     >     >> privileged
>     >
>     >     >>>>> material for the sole use of the intended recipient. Any
review,
>     >
>     >     >>> copying,
>     >
>     >     >>>>> or distribution of this email (or any attachments) by
others is
>     >
>     >     >>>> prohibited.
>     >
>     >     >>>>> If you are not the intended recipient, please contact the
sender
>     >
>     >     >>>>> immediately and permanently delete this email and any
attachments. No
>     >
>     >     >>>>> employee or agent of TiVo Inc. is authorized to conclude
any binding
>     >
>     >     >>>>> agreement on behalf of TiVo Inc. by email. Binding
agreements with
>     >
>     >     >> TiVo
>     >
>     >     >>>>> Inc. may only be made by a signed written agreement.
>     >
>     >     >>>>>
>     >
>     >     >>>>
>     >
>     >     >>>
>     >
>     >     >>
>     >
>     >     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     > --
>     >
>     > <http://www.openbet.com/>
>     >
>     > *Michal Borowiecki*
>     >
>     > *Senior Software Engineer L4*
>     >
>     > *T: *
>     >
>     > +44 208 742 1600 <+44%2020%208742%201600>
>     >
>     > +44 203 249 8448 <+44%2020%203249%208448>
>     >
>     >
>     >
>     > *E: *
>     >
>     > michal.borowiecki@openbet.com
>     >
>     > *W: *
>     >
>     > www.openbet.com
>     >
>     > *OpenBet Ltd*
>     >
>     > Chiswick Park Building 9
>     >
>     > 566 Chiswick High Rd
>     >
>     > London
>     >
>     > W4 5XT
>     >
>     > UK
>     >
>     > <https://www.openbet.com/email_promo>
>     >
>     >
>     >
>     > This message is confidential and intended only for the addressee.
If you
>     > have received this message in error, please immediately notify the
>     > postmaster@openbet.com and delete it from your system as well as any
>     > copies. The content of e-mails as well as traffic data may be
monitored by
>     > OpenBet for employment and security purposes. To protect the
environment
>     > please do not print this e-mail unless necessary. OpenBet Ltd.
Registered
>     > Office: Chiswick Park Building 9, 566 Chiswick High Road, London,
W4 5XT,
>     > United Kingdom. A company registered in England and Wales.
Registered no.
>     > 3134634. VAT no. GB927523612
>     >
>     >
>     >
>
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks for preparing this page!

About terminology:

You introduce the term "event time" -- but we should call this "stream
time" -- "stream time" is whatever TimestampExtractor returns and this
could be event time, ingestion time, or processing/wall-clock time.

Does this make sense to you?



-Matthias


On 4/10/17 4:58 AM, Arun Mathew wrote:
> Thanks Ewen.
> 
> @Michal, @all, I have created a child page to start the Use Cases discussion [https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases]. Please go through it and give your comments.
> 
> @Tianji, Sorry for the delay. I am trying to make the patch public.
> 
> --
> Arun Mathew
> 
> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> wrote:
> 
>     Arun,
>     
>     I've given you permission to edit the wiki. Let me know if you run into any
>     issues.
>     
>     -Ewen
>     
>     On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <am...@yahoo-corp.jp> wrote:
>     
>     > Thanks Michal. I don’t have the access yet [arunmathew88]. Should I be
>     > sending a separate mail for this?
>     >
>     > I thought one of the person following this thread would be able to give me
>     > access.
>     >
>     >
>     >
>     > *From: *Michal Borowiecki <mi...@openbet.com>
>     > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>     > *Date: *Friday, April 7, 2017 at 17:16
>     > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>     > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
>     >
>     >
>     >
>     > Hi Arun,
>     >
>     > I was thinking along the same lines as you, listing the use cases on the
>     > wiki, but didn't find time to get around doing that yet.
>     > Don't mind if you do it if you have access now.
>     > I was thinking it would be nice if, once we have the use cases listed,
>     > people could use likes to up-vote the use cases similar to what they're
>     > working on.
>     >
>     > I should have a bit more time to action this in the next few days, but
>     > happy for you to do it if you can beat me to it ;-)
>     >
>     > Cheers,
>     > Michal
>     >
>     > On 07/04/17 04:39, Arun Mathew wrote:
>     >
>     > Sure, Thanks Matthias. My id is [arunmathew88].
>     >
>     >
>     >
>     > Of course. I was thinking of a subpage where people can collaborate.
>     >
>     >
>     >
>     > Will do as per Michael’s suggestion.
>     >
>     >
>     >
>     > Regards,
>     >
>     > Arun Mathew
>     >
>     >
>     >
>     > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io> wrote:
>     >
>     >
>     >
>     >     Please share your Wiki-ID and a committer can give you write access.
>     >
>     >
>     >
>     >     Btw: as you did not initiate the KIP, you should not change the KIP
>     >
>     >     without the permission of the original author -- in this case Michael.
>     >
>     >
>     >
>     >     So you might also just share your thought over the mailing list and
>     >
>     >     Michael can update the KIP page. Or, as an alternative, just create a
>     >
>     >     subpage for the KIP page.
>     >
>     >
>     >
>     >     @Michael: WDYT?
>     >
>     >
>     >
>     >
>     >
>     >     -Matthias
>     >
>     >
>     >
>     >
>     >
>     >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>     >
>     >     > Hi Jay,
>     >
>     >     >           Thanks for the advise, I would like to list down the use cases as
>     >
>     >     > per your suggestion. But it seems I don't have write permission to the
>     >
>     >     > Apache Kafka Confluent Space. Whom shall I request for it?
>     >
>     >     >
>     >
>     >     > Regarding your last question. We are using a patch in our production system
>     >
>     >     > which does exactly this.
>     >
>     >     > We window by the event time, but trigger punctuate in <punctuate interval>
>     >
>     >     > duration of system time, in the absence of an event crossing the punctuate
>     >
>     >     > event time.
>     >
>     >     >
>     >
>     >     > We are using Kafka Stream for our Audit Trail, where we need to output the
>     >
>     >     > event counts on each topic on each cluster aggregated over a 1 minute
>     >
>     >     > window. We have to use event time to be able to cross check the counts. But
>     >
>     >     > we need to trigger punctuate [aggregate event pushes] by system time in the
>     >
>     >     > absence of events. Otherwise the event counts for unexpired windows would
>     >
>     >     > be 0 which is bad.
>     >
>     >     >
>     >
>     >     > "Maybe a hybrid solution works: I window by event time but trigger results
>     >
>     >     > by system time for windows that have updated? Not really sure the details
>     >
>     >     > of making that work. Does that work? Are there concrete examples where you
>     >
>     >     > actually want the current behavior?"
>     >
>     >     >
>     >
>     >     > --
>     >
>     >     > With Regards,
>     >
>     >     >
>     >
>     >     > Arun Mathew
>     >
>     >     > Yahoo! JAPAN Corporation
>     >
>     >     >
>     >
>     >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> <sk...@gmail.com> wrote:
>     >
>     >     >
>     >
>     >     >> Hi Jay,
>     >
>     >     >>
>     >
>     >     >> The hybrid solution is exactly what I expect and need for our use cases
>     >
>     >     >> when dealing with telecom data.
>     >
>     >     >>
>     >
>     >     >> Thanks
>     >
>     >     >> Tianji
>     >
>     >     >>
>     >
>     >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> <ja...@confluent.io> wrote:
>     >
>     >     >>
>     >
>     >     >>> Hey guys,
>     >
>     >     >>>
>     >
>     >     >>> One thing I've always found super important for this kind of design work
>     >
>     >     >> is
>     >
>     >     >>> to do a really good job of cataloging the landscape of use cases and how
>     >
>     >     >>> prevalent each one is. By that I mean not just listing lots of uses, but
>     >
>     >     >>> also grouping them into categories that functionally need the same thing.
>     >
>     >     >>> In the absence of this it is very hard to reason about design proposals.
>     >
>     >     >>> From the proposals so far I think we have a lot of discussion around
>     >
>     >     >>> possible apis, but less around what the user needs for different use
>     >
>     >     >> cases
>     >
>     >     >>> and how they would implement that using the api.
>     >
>     >     >>>
>     >
>     >     >>> Here is an example:
>     >
>     >     >>> You aggregate click and impression data for a reddit like site. Every ten
>     >
>     >     >>> minutes you want to output a ranked list of the top 10 articles ranked by
>     >
>     >     >>> clicks/impressions for each geographical area. I want to be able run this
>     >
>     >     >>> in steady state as well as rerun to regenerate results (or catch up if it
>     >
>     >     >>> crashes).
>     >
>     >     >>>
>     >
>     >     >>> There are a couple of tricky things that seem to make this hard with
>     >
>     >     >> either
>     >
>     >     >>> of the options proposed:
>     >
>     >     >>> 1. If I emit this data using event time I have the problem described
>     >
>     >     >> where
>     >
>     >     >>> a geographical region with no new clicks or impressions will fail to
>     >
>     >     >> output
>     >
>     >     >>> results.
>     >
>     >     >>> 2. If I emit this data using system time I have the problem that when
>     >
>     >     >>> reprocessing data my window may not be ten minutes but 10 hours if my
>     >
>     >     >>> processing is very fast so it dramatically changes the output.
>     >
>     >     >>>
>     >
>     >     >>> Maybe a hybrid solution works: I window by event time but trigger results
>     >
>     >     >>> by system time for windows that have updated? Not really sure the details
>     >
>     >     >>> of making that work. Does that work? Are there concrete examples where
>     >
>     >     >> you
>     >
>     >     >>> actually want the current behavior?
>     >
>     >     >>>
>     >
>     >     >>> -Jay
>     >
>     >     >>>
>     >
>     >     >>>
>     >
>     >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>     >
>     >     >>> wrote:
>     >
>     >     >>>
>     >
>     >     >>>> Hi All,
>     >
>     >     >>>>
>     >
>     >     >>>> Thanks for the KIP. We were also in need of a mechanism to trigger
>     >
>     >     >>>> punctuate in the absence of events.
>     >
>     >     >>>>
>     >
>     >     >>>> As I described in [
>     >
>     >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>     >
>     >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>     >
>     >     >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
>     >
>     >     >>>> ],
>     >
>     >     >>>>
>     >
>     >     >>>>    - Our approached involved using the event time by default.
>     >
>     >     >>>>    - The method to check if there is any punctuate ready in the
>     >
>     >     >>>>    PunctuationQueue is triggered via the any event received by the
>     >
>     >     >> stream
>     >
>     >     >>>>    tread, or at the polling intervals in the absence of any events.
>     >
>     >     >>>>    - When we create Punctuate objects (which contains the next event
>     >
>     >     >> time
>     >
>     >     >>>>    for punctuation and interval), we also record the creation time
>     >
>     >     >>> (system
>     >
>     >     >>>>    time).
>     >
>     >     >>>>    - While checking for maturity of Punctuate Schedule by
>     >
>     >     >> mayBePunctuate
>     >
>     >     >>>>    method, we also check if the system clock has elapsed the punctuate
>     >
>     >     >>>>    interval since the schedule creation time.
>     >
>     >     >>>>    - In the absence of any event, or in the absence of any event for
>     >
>     >     >> one
>     >
>     >     >>>>    topic in the partition group assigned to the stream task, the system
>     >
>     >     >>>> time
>     >
>     >     >>>>    will elapse the interval and we trigger a punctuate using the
>     >
>     >     >> expected
>     >
>     >     >>>>    punctuation event time.
>     >
>     >     >>>>    - we then create the next punctuation schedule as punctuation event
>     >
>     >     >>> time
>     >
>     >     >>>>    + punctuation interval, [again recording the system time of creation
>     >
>     >     >>> of
>     >
>     >     >>>> the
>     >
>     >     >>>>    schedule].
>     >
>     >     >>>>
>     >
>     >     >>>> We call this a Hybrid Punctuate. Of course, this approach has pros and
>     >
>     >     >>>> cons.
>     >
>     >     >>>> Pros
>     >
>     >     >>>>
>     >
>     >     >>>>    - Punctuates will happen in <punctuate interval> time duration at
>     >
>     >     >> max
>     >
>     >     >>> in
>     >
>     >     >>>>    terms of system time.
>     >
>     >     >>>>    - The semantics as a whole continues to revolve around event time.
>     >
>     >     >>>>    - We can use the old data [old timestamps] to rerun any experiments
>     >
>     >     >> or
>     >
>     >     >>>>    tests.
>     >
>     >     >>>>
>     >
>     >     >>>> Cons
>     >
>     >     >>>>
>     >
>     >     >>>>    - In case the  <punctuate interval> is not a time duration [say
>     >
>     >     >>> logical
>     >
>     >     >>>>    time/event count], then the approach might not be meaningful.
>     >
>     >     >>>>    - In case there is a case where we have to wait for an actual event
>     >
>     >     >>> from
>     >
>     >     >>>>    a low event rate partition in the partition group, this approach
>     >
>     >     >> will
>     >
>     >     >>>> jump
>     >
>     >     >>>>    the gun.
>     >
>     >     >>>>    - in case the event processing cannot catch up with the event rate
>     >
>     >     >> and
>     >
>     >     >>>>    the expected timestamp events gets queued for long time, this
>     >
>     >     >> approach
>     >
>     >     >>>>    might jump the gun.
>     >
>     >     >>>>
>     >
>     >     >>>> I believe the above approach and discussion goes close to the approach
>     >
>     >     >> A.
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I like the idea of having an even count based punctuate.
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I agree with the discussion around approach C, that we should provide
>     >
>     >     >> the
>     >
>     >     >>>> user with the option to choose system time or event time based
>     >
>     >     >>> punctuates.
>     >
>     >     >>>> But I believe that the user predominantly wants to use event time while
>     >
>     >     >>> not
>     >
>     >     >>>> missing out on regular punctuates due to event delays or event
>     >
>     >     >> absences.
>     >
>     >     >>>> Hence a complex punctuate option as Matthias mentioned (quoted below)
>     >
>     >     >>> would
>     >
>     >     >>>> be most apt.
>     >
>     >     >>>>
>     >
>     >     >>>> "- We might want to add "complex" schedules later on (like, punctuate
>     >
>     >     >> on
>     >
>     >     >>>> every 10 seconds event-time or 60 seconds system-time whatever comes
>     >
>     >     >>>> first)."
>     >
>     >     >>>>
>     >
>     >     >>>> -----------
>     >
>     >     >>>>
>     >
>     >     >>>> I think I read somewhere that Kafka Streams started with System Time as
>     >
>     >     >>> the
>     >
>     >     >>>> punctuation standard, but was later changed to Event Time. I guess
>     >
>     >     >> there
>     >
>     >     >>>> would be some good reason behind it. As Kafka Streams want to evolve
>     >
>     >     >> more
>     >
>     >     >>>> on the Stream Processing front, I believe the emphasis on event time
>     >
>     >     >>> would
>     >
>     >     >>>> remain quite strong.
>     >
>     >     >>>>
>     >
>     >     >>>>
>     >
>     >     >>>> With Regards,
>     >
>     >     >>>>
>     >
>     >     >>>> Arun Mathew
>     >
>     >     >>>> Yahoo! JAPAN Corporation, Tokyo
>     >
>     >     >>>>
>     >
>     >     >>>>
>     >
>     >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com> <to...@tivo.com>
>     >
>     >     >> wrote:
>     >
>     >     >>>>
>     >
>     >     >>>>> Yeah I like PuncutationType much better; I just threw Time out there
>     >
>     >     >>>>> more as a strawman than an actual suggestion ;) I still think it's
>     >
>     >     >>>>> worth considering what this buys us over an additional callback. I
>     >
>     >     >>>>> foresee a number of punctuate implementations following this pattern:
>     >
>     >     >>>>>
>     >
>     >     >>>>> public void punctuate(PunctuationType type) {
>     >
>     >     >>>>>     switch (type) {
>     >
>     >     >>>>>         case EVENT_TIME:
>     >
>     >     >>>>>             methodA();
>     >
>     >     >>>>>             break;
>     >
>     >     >>>>>         case SYSTEM_TIME:
>     >
>     >     >>>>>             methodB();
>     >
>     >     >>>>>             break;
>     >
>     >     >>>>>     }
>     >
>     >     >>>>> }
>     >
>     >     >>>>>
>     >
>     >     >>>>> I guess one advantage of this approach is we could add additional
>     >
>     >     >>>>> punctuation types later in a backwards compatible way (like event
>     >
>     >     >> count
>     >
>     >     >>>>> as you mentioned).
>     >
>     >     >>>>>
>     >
>     >     >>>>> -Tommy
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
>     >
>     >     >>>>>> That sounds promising.
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> I am just wondering if `Time` is the best name. Maybe we want to
>     >
>     >     >> add
>     >
>     >     >>>>>> other non-time based punctuations at some point later. I would
>     >
>     >     >>>>>> suggest
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> enum PunctuationType {
>     >
>     >     >>>>>>   EVENT_TIME,
>     >
>     >     >>>>>>   SYSTEM_TIME,
>     >
>     >     >>>>>> }
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> or similar. Just to keep the door open -- it's easier to add new
>     >
>     >     >>>>>> stuff
>     >
>     >     >>>>>> if the name is more generic.
>     >
>     >     >>>>>>
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> -Matthias
>     >
>     >     >>>>>>
>     >
>     >     >>>>>>
>     >
>     >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> I agree that the framework providing and managing the notion of
>     >
>     >     >>>>>>> stream
>     >
>     >     >>>>>>> time is valuable and not something we would want to delegate to
>     >
>     >     >> the
>     >
>     >     >>>>>>> tasks. I'm not entirely convinced that a separate callback
>     >
>     >     >> (option
>     >
>     >     >>>>>>> C)
>     >
>     >     >>>>>>> is that messy (it could just be a default method with an empty
>     >
>     >     >>>>>>> implementation), but if we wanted a single API to handle both
>     >
>     >     >>>>>>> cases,
>     >
>     >     >>>>>>> how about something like the following?
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> enum Time {
>     >
>     >     >>>>>>>    STREAM,
>     >
>     >     >>>>>>>    CLOCK
>     >
>     >     >>>>>>> }
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Then on ProcessorContext:
>     >
>     >     >>>>>>> context.schedule(Time time, long interval)  // We could allow
>     >
>     >     >> this
>     >
>     >     >>>>>>> to
>     >
>     >     >>>>>>> be called once for each value of time to mix approaches.
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Then the Processor API becomes:
>     >
>     >     >>>>>>> punctuate(Time time) // time here denotes which schedule resulted
>     >
>     >     >>>>>>> in
>     >
>     >     >>>>>>> this call.
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> Thoughts?
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Thanks a lot for the KIP Michal,
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> I was thinking about the four options you proposed in more
>     >
>     >     >>>>>>>> details
>     >
>     >     >>>>>>>> and
>     >
>     >     >>>>>>>> this are my thoughts:
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (A) You argue, that users can still "punctuate" on event-time
>     >
>     >     >> via
>     >
>     >     >>>>>>>> process(), but I am not sure if this is possible. Note, that
>     >
>     >     >>>>>>>> users
>     >
>     >     >>>>>>>> only
>     >
>     >     >>>>>>>> get record timestamps via context.timestamp(). Thus, users
>     >
>     >     >> would
>     >
>     >     >>>>>>>> need
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> track the time progress per partition (based on the partitions
>     >
>     >     >>>>>>>> they
>     >
>     >     >>>>>>>> obverse via context.partition(). (This alone puts a huge burden
>     >
>     >     >>>>>>>> on
>     >
>     >     >>>>>>>> the
>     >
>     >     >>>>>>>> user by itself.) However, users are not notified at startup
>     >
>     >     >> what
>     >
>     >     >>>>>>>> partitions are assigned, and user are not notified when
>     >
>     >     >>>>>>>> partitions
>     >
>     >     >>>>>>>> get
>     >
>     >     >>>>>>>> revoked. Because this information is not available, it's not
>     >
>     >     >>>>>>>> possible
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> "manually advance" stream-time, and thus event-time punctuation
>     >
>     >     >>>>>>>> within
>     >
>     >     >>>>>>>> process() seems not to be possible -- or do you see a way to
>     >
>     >     >> get
>     >
>     >     >>>>>>>> it
>     >
>     >     >>>>>>>> done? And even if, it might still be too clumsy to use.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (B) This does not allow to mix both approaches, thus limiting
>     >
>     >     >>>>>>>> what
>     >
>     >     >>>>>>>> users
>     >
>     >     >>>>>>>> can do.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (C) This should give all flexibility we need. However, just
>     >
>     >     >>>>>>>> adding
>     >
>     >     >>>>>>>> one
>     >
>     >     >>>>>>>> more method seems to be a solution that is too simple (cf my
>     >
>     >     >>>>>>>> comments
>     >
>     >     >>>>>>>> below).
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> (D) This might be hard to use. Also, I am not sure how a user
>     >
>     >     >>>>>>>> could
>     >
>     >     >>>>>>>> enable system-time and event-time punctuation in parallel.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Overall options (C) seems to be the most promising approach to
>     >
>     >     >>>>>>>> me.
>     >
>     >     >>>>>>>> Because I also favor a clean API, we might keep current
>     >
>     >     >>>>>>>> punctuate()
>     >
>     >     >>>>>>>> as-is, but deprecate it -- so we can remove it at some later
>     >
>     >     >>>>>>>> point
>     >
>     >     >>>>>>>> when
>     >
>     >     >>>>>>>> people use the "new punctuate API".
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> Couple of follow up questions:
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - I am wondering, if we should have two callback methods or
>     >
>     >     >> just
>     >
>     >     >>>>>>>> one
>     >
>     >     >>>>>>>> (ie, a unified for system and event time punctuation or one for
>     >
>     >     >>>>>>>> each?).
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - If we have one, how can the user figure out, which condition
>     >
>     >     >>>>>>>> did
>     >
>     >     >>>>>>>> trigger?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - How would the API look like, for registering different
>     >
>     >     >>>>>>>> punctuate
>     >
>     >     >>>>>>>> schedules? The "type" must be somehow defined?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - We might want to add "complex" schedules later on (like,
>     >
>     >     >>>>>>>> punctuate
>     >
>     >     >>>>>>>> on
>     >
>     >     >>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever
>     >
>     >     >>>>>>>> comes
>     >
>     >     >>>>>>>> first). I don't say we should add this right away, but we might
>     >
>     >     >>>>>>>> want
>     >
>     >     >>>>>>>> to
>     >
>     >     >>>>>>>> define the API in a way, that it allows extensions like this
>     >
>     >     >>>>>>>> later
>     >
>     >     >>>>>>>> on,
>     >
>     >     >>>>>>>> without redesigning the API (ie, the API should be designed
>     >
>     >     >>>>>>>> extensible)
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> - Did you ever consider count-based punctuation?
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> I understand, that you would like to solve a simple problem,
>     >
>     >     >> but
>     >
>     >     >>>>>>>> we
>     >
>     >     >>>>>>>> learned from the past, that just "adding some API" quickly
>     >
>     >     >> leads
>     >
>     >     >>>>>>>> to a
>     >
>     >     >>>>>>>> not very well defined API that needs time consuming clean up
>     >
>     >     >>>>>>>> later on
>     >
>     >     >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
>     >
>     >     >>>>>>>> punctuation
>     >
>     >     >>>>>>>> KIP
>     >
>     >     >>>>>>>> with this from the beginning on to avoid later painful
>     >
>     >     >> redesign.
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> -Matthias
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>>
>     >
>     >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Thanks Thomas,
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> I'm also wary of changing the existing semantics of
>     >
>     >     >> punctuate,
>     >
>     >     >>>>>>>>> for
>     >
>     >     >>>>>>>>> backward compatibility reasons, although I like the
>     >
>     >     >> conceptual
>     >
>     >     >>>>>>>>> simplicity of that option.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.
>     >
>     >     >> I
>     >
>     >     >>>>>>>>> added
>     >
>     >     >>>>>>>>> this to the KIP now as option (C).
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> The TimestampExtractor mechanism is actually more flexible,
>     >
>     >     >> as
>     >
>     >     >>>>>>>>> it
>     >
>     >     >>>>>>>>> allows
>     >
>     >     >>>>>>>>> you to return any value, you're not limited to event time or
>     >
>     >     >>>>>>>>> system
>     >
>     >     >>>>>>>>> time
>     >
>     >     >>>>>>>>> (although I don't see an actual use case where you might need
>     >
>     >     >>>>>>>>> anything
>     >
>     >     >>>>>>>>> else then those two). Hence I also proposed the option to
>     >
>     >     >> allow
>     >
>     >     >>>>>>>>> users
>     >
>     >     >>>>>>>>> to, effectively, decide what "stream time" is for them given
>     >
>     >     >>>>>>>>> the
>     >
>     >     >>>>>>>>> presence or absence of messages, much like they can decide
>     >
>     >     >> what
>     >
>     >     >>>>>>>>> msg
>     >
>     >     >>>>>>>>> time
>     >
>     >     >>>>>>>>> means for them using the TimestampExtractor. What do you
>     >
>     >     >> think
>     >
>     >     >>>>>>>>> about
>     >
>     >     >>>>>>>>> that? This is probably most flexible but also most
>     >
>     >     >> complicated.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> All comments appreciated.
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Cheers,
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> Michal
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>>
>     >
>     >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>> Although I fully agree we need a way to trigger periodic
>     >
>     >     >>>>>>>>>> processing
>     >
>     >     >>>>>>>>>> that is independent from whether and when messages arrive,
>     >
>     >     >>>>>>>>>> I'm
>     >
>     >     >>>>>>>>>> not sure
>     >
>     >     >>>>>>>>>> I like the idea of changing the existing semantics across
>     >
>     >     >> the
>     >
>     >     >>>>>>>>>> board.
>     >
>     >     >>>>>>>>>> What if we added an additional callback to Processor that
>     >
>     >     >> can
>     >
>     >     >>>>>>>>>> be
>     >
>     >     >>>>>>>>>> scheduled similarly to punctuate() but was always called at
>     >
>     >     >>>>>>>>>> fixed, wall
>     >
>     >     >>>>>>>>>> clock based intervals? This way you wouldn't have to give
>     >
>     >     >> up
>     >
>     >     >>>>>>>>>> the
>     >
>     >     >>>>>>>>>> notion
>     >
>     >     >>>>>>>>>> of stream time to be able to do periodic processing.
>     >
>     >     >>>>>>>>>>
>     >
>     >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Hi all,
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
>     >
>     >     >>>>>>>>>>> semantics
>     >
>     >     >>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP- <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>     >
>     >     > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>> 138%
>     >
>     >     >>>>>>>>>>> 3A+C
>     >
>     >     >>>>>>>>>>> hange+
>     >
>     >     >>>>>>>>>>> punctuate+semantics>
>     >
>     >     >>>>>>>>>>> .
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Appreciating there can be different views on system-time
>     >
>     >     >> vs
>     >
>     >     >>>>>>>>>>> event-
>     >
>     >     >>>>>>>>>>> time
>     >
>     >     >>>>>>>>>>> semantics for punctuation depending on use-case and the
>     >
>     >     >>>>>>>>>>> importance of
>     >
>     >     >>>>>>>>>>> backwards compatibility of any such change, I've left it
>     >
>     >     >>>>>>>>>>> quite
>     >
>     >     >>>>>>>>>>> open
>     >
>     >     >>>>>>>>>>> and
>     >
>     >     >>>>>>>>>>> hope to fill in more info as the discussion progresses.
>     >
>     >     >>>>>>>>>>>
>     >
>     >     >>>>>>>>>>> Thanks,
>     >
>     >     >>>>>>>>>>> Michal
>     >
>     >     >>>>>>> --
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     Tommy Becker
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     Senior Software Engineer
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>     tivo.com
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> ________________________________
>     >
>     >     >>>>>>>
>     >
>     >     >>>>>>> This email and any attachments may contain confidential and
>     >
>     >     >>>>>>> privileged material for the sole use of the intended recipient.
>     >
>     >     >> Any
>     >
>     >     >>>>>>> review, copying, or distribution of this email (or any
>     >
>     >     >> attachments)
>     >
>     >     >>>>>>> by others is prohibited. If you are not the intended recipient,
>     >
>     >     >>>>>>> please contact the sender immediately and permanently delete this
>     >
>     >     >>>>>>> email and any attachments. No employee or agent of TiVo Inc. is
>     >
>     >     >>>>>>> authorized to conclude any binding agreement on behalf of TiVo
>     >
>     >     >> Inc.
>     >
>     >     >>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a
>     >
>     >     >>>>>>> signed written agreement.
>     >
>     >     >>>>>>>
>     >
>     >     >>>>> --
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>>     Tommy Becker
>     >
>     >     >>>>>
>     >
>     >     >>>>>     Senior Software Engineer
>     >
>     >     >>>>>
>     >
>     >     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
>     >
>     >     >>>>>
>     >
>     >     >>>>>     tivo.com
>     >
>     >     >>>>>
>     >
>     >     >>>>>
>     >
>     >     >>>>> ________________________________
>     >
>     >     >>>>>
>     >
>     >     >>>>> This email and any attachments may contain confidential and
>     >
>     >     >> privileged
>     >
>     >     >>>>> material for the sole use of the intended recipient. Any review,
>     >
>     >     >>> copying,
>     >
>     >     >>>>> or distribution of this email (or any attachments) by others is
>     >
>     >     >>>> prohibited.
>     >
>     >     >>>>> If you are not the intended recipient, please contact the sender
>     >
>     >     >>>>> immediately and permanently delete this email and any attachments. No
>     >
>     >     >>>>> employee or agent of TiVo Inc. is authorized to conclude any binding
>     >
>     >     >>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with
>     >
>     >     >> TiVo
>     >
>     >     >>>>> Inc. may only be made by a signed written agreement.
>     >
>     >     >>>>>
>     >
>     >     >>>>
>     >
>     >     >>>
>     >
>     >     >>
>     >
>     >     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     >
>     > --
>     >
>     > <http://www.openbet.com/>
>     >
>     > *Michal Borowiecki*
>     >
>     > *Senior Software Engineer L4*
>     >
>     > *T: *
>     >
>     > +44 208 742 1600 <+44%2020%208742%201600>
>     >
>     > +44 203 249 8448 <+44%2020%203249%208448>
>     >
>     >
>     >
>     > *E: *
>     >
>     > michal.borowiecki@openbet.com
>     >
>     > *W: *
>     >
>     > www.openbet.com
>     >
>     > *OpenBet Ltd*
>     >
>     > Chiswick Park Building 9
>     >
>     > 566 Chiswick High Rd
>     >
>     > London
>     >
>     > W4 5XT
>     >
>     > UK
>     >
>     > <https://www.openbet.com/email_promo>
>     >
>     >
>     >
>     > This message is confidential and intended only for the addressee. If you
>     > have received this message in error, please immediately notify the
>     > postmaster@openbet.com and delete it from your system as well as any
>     > copies. The content of e-mails as well as traffic data may be monitored by
>     > OpenBet for employment and security purposes. To protect the environment
>     > please do not print this e-mail unless necessary. OpenBet Ltd. Registered
>     > Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
>     > United Kingdom. A company registered in England and Wales. Registered no.
>     > 3134634. VAT no. GB927523612
>     >
>     >
>     >
>     
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi Arun,

Thanks for putting the use cases on the wiki. I copied over your 
Terminology section to the main KIP page as I think it's super important 
to be clear on the terms.

I've made some changes while doing that which I highlight below, as I'd 
like to encourage comments on these.

1) I removed the mention of logical time, since the API is strictly 
mandates " milliseconds since midnight, January 1, 1970 UTC"  as opposed 
to any arbitrary logical time (even if it's not enforceable).

2) I broke up the definition of Stream Time into 2 separate terms: 
Stream Partition Time and Stream Time proper. This is for 2 reasons:

a) Follows the definition of Stream Time as it is stated on the 
ProcessorContext: 
https://github.com/apache/kafka/blob/0.10.2.0/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java#L159

b) The timestamp extractors are stealing all the thunder ;-)
There's been a lot of discussion about timestamp extractors and merits 
of event/processing time, however I haven't encountered much in terms of 
justification why the stream time is fixed to be the /_smallest_/ among 
all its input stream partition timestamps. I found a comment in the 
PartitionGroup: 
https://github.com/apache/kafka/blob/0.10.2.0/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java#L138
> public long timestamp() {
> // we should always return the smallest timestamp of all partitions
> // to avoid group partition time goes backward
>
but I can't believe this to be the only reason behind this choice as 
minimum is not the only function to guarantee the group partition time 
never going back. Using the largest or the average among partitions' 
timestamp would also guaranteed the group timestamp not going back as 
timestamp never goes back for any individual partition.
So why was minimum chosen? Is it depended on by window semantics 
somewhere or anything else?

3) I used the term Punctuate'stimestampargument instead of Punctuation 
Timestamp since I found the latter sound too similar to Punctuate Time

4) Rephrased Output Record Time. This is something I haven't given any 
thought before whatsoever. Is it still true to what you meant?


Comments appreciated, especially need input on 2b above.

Cheers,
Michal


On 10/04/17 12:58, Arun Mathew wrote:
> Thanks Ewen.
>
> @Michal, @all, I have created a child page to start the Use Cases discussion [https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases]. Please go through it and give your comments.
>
> @Tianji, Sorry for the delay. I am trying to make the patch public.
>
> --
> Arun Mathew
>
> On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> wrote:
>
>      Arun,
>      
>      I've given you permission to edit the wiki. Let me know if you run into any
>      issues.
>      
>      -Ewen
>      
>      On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <am...@yahoo-corp.jp> wrote:
>      
>      > Thanks Michal. I don\u2019t have the access yet [arunmathew88]. Should I be
>      > sending a separate mail for this?
>      >
>      > I thought one of the person following this thread would be able to give me
>      > access.
>      >
>      >
>      >
>      > *From: *Michal Borowiecki <mi...@openbet.com>
>      > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>      > *Date: *Friday, April 7, 2017 at 17:16
>      > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
>      > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
>      >
>      >
>      >
>      > Hi Arun,
>      >
>      > I was thinking along the same lines as you, listing the use cases on the
>      > wiki, but didn't find time to get around doing that yet.
>      > Don't mind if you do it if you have access now.
>      > I was thinking it would be nice if, once we have the use cases listed,
>      > people could use likes to up-vote the use cases similar to what they're
>      > working on.
>      >
>      > I should have a bit more time to action this in the next few days, but
>      > happy for you to do it if you can beat me to it ;-)
>      >
>      > Cheers,
>      > Michal
>      >
>      > On 07/04/17 04:39, Arun Mathew wrote:
>      >
>      > Sure, Thanks Matthias. My id is [arunmathew88].
>      >
>      >
>      >
>      > Of course. I was thinking of a subpage where people can collaborate.
>      >
>      >
>      >
>      > Will do as per Michael\u2019s suggestion.
>      >
>      >
>      >
>      > Regards,
>      >
>      > Arun Mathew
>      >
>      >
>      >
>      > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io> wrote:
>      >
>      >
>      >
>      >     Please share your Wiki-ID and a committer can give you write access.
>      >
>      >
>      >
>      >     Btw: as you did not initiate the KIP, you should not change the KIP
>      >
>      >     without the permission of the original author -- in this case Michael.
>      >
>      >
>      >
>      >     So you might also just share your thought over the mailing list and
>      >
>      >     Michael can update the KIP page. Or, as an alternative, just create a
>      >
>      >     subpage for the KIP page.
>      >
>      >
>      >
>      >     @Michael: WDYT?
>      >
>      >
>      >
>      >
>      >
>      >     -Matthias
>      >
>      >
>      >
>      >
>      >
>      >     On 4/6/17 8:05 PM, Arun Mathew wrote:
>      >
>      >     > Hi Jay,
>      >
>      >     >           Thanks for the advise, I would like to list down the use cases as
>      >
>      >     > per your suggestion. But it seems I don't have write permission to the
>      >
>      >     > Apache Kafka Confluent Space. Whom shall I request for it?
>      >
>      >     >
>      >
>      >     > Regarding your last question. We are using a patch in our production system
>      >
>      >     > which does exactly this.
>      >
>      >     > We window by the event time, but trigger punctuate in <punctuate interval>
>      >
>      >     > duration of system time, in the absence of an event crossing the punctuate
>      >
>      >     > event time.
>      >
>      >     >
>      >
>      >     > We are using Kafka Stream for our Audit Trail, where we need to output the
>      >
>      >     > event counts on each topic on each cluster aggregated over a 1 minute
>      >
>      >     > window. We have to use event time to be able to cross check the counts. But
>      >
>      >     > we need to trigger punctuate [aggregate event pushes] by system time in the
>      >
>      >     > absence of events. Otherwise the event counts for unexpired windows would
>      >
>      >     > be 0 which is bad.
>      >
>      >     >
>      >
>      >     > "Maybe a hybrid solution works: I window by event time but trigger results
>      >
>      >     > by system time for windows that have updated? Not really sure the details
>      >
>      >     > of making that work. Does that work? Are there concrete examples where you
>      >
>      >     > actually want the current behavior?"
>      >
>      >     >
>      >
>      >     > --
>      >
>      >     > With Regards,
>      >
>      >     >
>      >
>      >     > Arun Mathew
>      >
>      >     > Yahoo! JAPAN Corporation
>      >
>      >     >
>      >
>      >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> <sk...@gmail.com> wrote:
>      >
>      >     >
>      >
>      >     >> Hi Jay,
>      >
>      >     >>
>      >
>      >     >> The hybrid solution is exactly what I expect and need for our use cases
>      >
>      >     >> when dealing with telecom data.
>      >
>      >     >>
>      >
>      >     >> Thanks
>      >
>      >     >> Tianji
>      >
>      >     >>
>      >
>      >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> <ja...@confluent.io> wrote:
>      >
>      >     >>
>      >
>      >     >>> Hey guys,
>      >
>      >     >>>
>      >
>      >     >>> One thing I've always found super important for this kind of design work
>      >
>      >     >> is
>      >
>      >     >>> to do a really good job of cataloging the landscape of use cases and how
>      >
>      >     >>> prevalent each one is. By that I mean not just listing lots of uses, but
>      >
>      >     >>> also grouping them into categories that functionally need the same thing.
>      >
>      >     >>> In the absence of this it is very hard to reason about design proposals.
>      >
>      >     >>> From the proposals so far I think we have a lot of discussion around
>      >
>      >     >>> possible apis, but less around what the user needs for different use
>      >
>      >     >> cases
>      >
>      >     >>> and how they would implement that using the api.
>      >
>      >     >>>
>      >
>      >     >>> Here is an example:
>      >
>      >     >>> You aggregate click and impression data for a reddit like site. Every ten
>      >
>      >     >>> minutes you want to output a ranked list of the top 10 articles ranked by
>      >
>      >     >>> clicks/impressions for each geographical area. I want to be able run this
>      >
>      >     >>> in steady state as well as rerun to regenerate results (or catch up if it
>      >
>      >     >>> crashes).
>      >
>      >     >>>
>      >
>      >     >>> There are a couple of tricky things that seem to make this hard with
>      >
>      >     >> either
>      >
>      >     >>> of the options proposed:
>      >
>      >     >>> 1. If I emit this data using event time I have the problem described
>      >
>      >     >> where
>      >
>      >     >>> a geographical region with no new clicks or impressions will fail to
>      >
>      >     >> output
>      >
>      >     >>> results.
>      >
>      >     >>> 2. If I emit this data using system time I have the problem that when
>      >
>      >     >>> reprocessing data my window may not be ten minutes but 10 hours if my
>      >
>      >     >>> processing is very fast so it dramatically changes the output.
>      >
>      >     >>>
>      >
>      >     >>> Maybe a hybrid solution works: I window by event time but trigger results
>      >
>      >     >>> by system time for windows that have updated? Not really sure the details
>      >
>      >     >>> of making that work. Does that work? Are there concrete examples where
>      >
>      >     >> you
>      >
>      >     >>> actually want the current behavior?
>      >
>      >     >>>
>      >
>      >     >>> -Jay
>      >
>      >     >>>
>      >
>      >     >>>
>      >
>      >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>      >
>      >     >>> wrote:
>      >
>      >     >>>
>      >
>      >     >>>> Hi All,
>      >
>      >     >>>>
>      >
>      >     >>>> Thanks for the KIP. We were also in need of a mechanism to trigger
>      >
>      >     >>>> punctuate in the absence of events.
>      >
>      >     >>>>
>      >
>      >     >>>> As I described in [
>      >
>      >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>      >
>      >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>      >
>      >     >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
>      >
>      >     >>>> ],
>      >
>      >     >>>>
>      >
>      >     >>>>    - Our approached involved using the event time by default.
>      >
>      >     >>>>    - The method to check if there is any punctuate ready in the
>      >
>      >     >>>>    PunctuationQueue is triggered via the any event received by the
>      >
>      >     >> stream
>      >
>      >     >>>>    tread, or at the polling intervals in the absence of any events.
>      >
>      >     >>>>    - When we create Punctuate objects (which contains the next event
>      >
>      >     >> time
>      >
>      >     >>>>    for punctuation and interval), we also record the creation time
>      >
>      >     >>> (system
>      >
>      >     >>>>    time).
>      >
>      >     >>>>    - While checking for maturity of Punctuate Schedule by
>      >
>      >     >> mayBePunctuate
>      >
>      >     >>>>    method, we also check if the system clock has elapsed the punctuate
>      >
>      >     >>>>    interval since the schedule creation time.
>      >
>      >     >>>>    - In the absence of any event, or in the absence of any event for
>      >
>      >     >> one
>      >
>      >     >>>>    topic in the partition group assigned to the stream task, the system
>      >
>      >     >>>> time
>      >
>      >     >>>>    will elapse the interval and we trigger a punctuate using the
>      >
>      >     >> expected
>      >
>      >     >>>>    punctuation event time.
>      >
>      >     >>>>    - we then create the next punctuation schedule as punctuation event
>      >
>      >     >>> time
>      >
>      >     >>>>    + punctuation interval, [again recording the system time of creation
>      >
>      >     >>> of
>      >
>      >     >>>> the
>      >
>      >     >>>>    schedule].
>      >
>      >     >>>>
>      >
>      >     >>>> We call this a Hybrid Punctuate. Of course, this approach has pros and
>      >
>      >     >>>> cons.
>      >
>      >     >>>> Pros
>      >
>      >     >>>>
>      >
>      >     >>>>    - Punctuates will happen in <punctuate interval> time duration at
>      >
>      >     >> max
>      >
>      >     >>> in
>      >
>      >     >>>>    terms of system time.
>      >
>      >     >>>>    - The semantics as a whole continues to revolve around event time.
>      >
>      >     >>>>    - We can use the old data [old timestamps] to rerun any experiments
>      >
>      >     >> or
>      >
>      >     >>>>    tests.
>      >
>      >     >>>>
>      >
>      >     >>>> Cons
>      >
>      >     >>>>
>      >
>      >     >>>>    - In case the  <punctuate interval> is not a time duration [say
>      >
>      >     >>> logical
>      >
>      >     >>>>    time/event count], then the approach might not be meaningful.
>      >
>      >     >>>>    - In case there is a case where we have to wait for an actual event
>      >
>      >     >>> from
>      >
>      >     >>>>    a low event rate partition in the partition group, this approach
>      >
>      >     >> will
>      >
>      >     >>>> jump
>      >
>      >     >>>>    the gun.
>      >
>      >     >>>>    - in case the event processing cannot catch up with the event rate
>      >
>      >     >> and
>      >
>      >     >>>>    the expected timestamp events gets queued for long time, this
>      >
>      >     >> approach
>      >
>      >     >>>>    might jump the gun.
>      >
>      >     >>>>
>      >
>      >     >>>> I believe the above approach and discussion goes close to the approach
>      >
>      >     >> A.
>      >
>      >     >>>>
>      >
>      >     >>>> -----------
>      >
>      >     >>>>
>      >
>      >     >>>> I like the idea of having an even count based punctuate.
>      >
>      >     >>>>
>      >
>      >     >>>> -----------
>      >
>      >     >>>>
>      >
>      >     >>>> I agree with the discussion around approach C, that we should provide
>      >
>      >     >> the
>      >
>      >     >>>> user with the option to choose system time or event time based
>      >
>      >     >>> punctuates.
>      >
>      >     >>>> But I believe that the user predominantly wants to use event time while
>      >
>      >     >>> not
>      >
>      >     >>>> missing out on regular punctuates due to event delays or event
>      >
>      >     >> absences.
>      >
>      >     >>>> Hence a complex punctuate option as Matthias mentioned (quoted below)
>      >
>      >     >>> would
>      >
>      >     >>>> be most apt.
>      >
>      >     >>>>
>      >
>      >     >>>> "- We might want to add "complex" schedules later on (like, punctuate
>      >
>      >     >> on
>      >
>      >     >>>> every 10 seconds event-time or 60 seconds system-time whatever comes
>      >
>      >     >>>> first)."
>      >
>      >     >>>>
>      >
>      >     >>>> -----------
>      >
>      >     >>>>
>      >
>      >     >>>> I think I read somewhere that Kafka Streams started with System Time as
>      >
>      >     >>> the
>      >
>      >     >>>> punctuation standard, but was later changed to Event Time. I guess
>      >
>      >     >> there
>      >
>      >     >>>> would be some good reason behind it. As Kafka Streams want to evolve
>      >
>      >     >> more
>      >
>      >     >>>> on the Stream Processing front, I believe the emphasis on event time
>      >
>      >     >>> would
>      >
>      >     >>>> remain quite strong.
>      >
>      >     >>>>
>      >
>      >     >>>>
>      >
>      >     >>>> With Regards,
>      >
>      >     >>>>
>      >
>      >     >>>> Arun Mathew
>      >
>      >     >>>> Yahoo! JAPAN Corporation, Tokyo
>      >
>      >     >>>>
>      >
>      >     >>>>
>      >
>      >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com> <to...@tivo.com>
>      >
>      >     >> wrote:
>      >
>      >     >>>>
>      >
>      >     >>>>> Yeah I like PuncutationType much better; I just threw Time out there
>      >
>      >     >>>>> more as a strawman than an actual suggestion ;) I still think it's
>      >
>      >     >>>>> worth considering what this buys us over an additional callback. I
>      >
>      >     >>>>> foresee a number of punctuate implementations following this pattern:
>      >
>      >     >>>>>
>      >
>      >     >>>>> public void punctuate(PunctuationType type) {
>      >
>      >     >>>>>     switch (type) {
>      >
>      >     >>>>>         case EVENT_TIME:
>      >
>      >     >>>>>             methodA();
>      >
>      >     >>>>>             break;
>      >
>      >     >>>>>         case SYSTEM_TIME:
>      >
>      >     >>>>>             methodB();
>      >
>      >     >>>>>             break;
>      >
>      >     >>>>>     }
>      >
>      >     >>>>> }
>      >
>      >     >>>>>
>      >
>      >     >>>>> I guess one advantage of this approach is we could add additional
>      >
>      >     >>>>> punctuation types later in a backwards compatible way (like event
>      >
>      >     >> count
>      >
>      >     >>>>> as you mentioned).
>      >
>      >     >>>>>
>      >
>      >     >>>>> -Tommy
>      >
>      >     >>>>>
>      >
>      >     >>>>>
>      >
>      >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
>      >
>      >     >>>>>> That sounds promising.
>      >
>      >     >>>>>>
>      >
>      >     >>>>>> I am just wondering if `Time` is the best name. Maybe we want to
>      >
>      >     >> add
>      >
>      >     >>>>>> other non-time based punctuations at some point later. I would
>      >
>      >     >>>>>> suggest
>      >
>      >     >>>>>>
>      >
>      >     >>>>>> enum PunctuationType {
>      >
>      >     >>>>>>   EVENT_TIME,
>      >
>      >     >>>>>>   SYSTEM_TIME,
>      >
>      >     >>>>>> }
>      >
>      >     >>>>>>
>      >
>      >     >>>>>> or similar. Just to keep the door open -- it's easier to add new
>      >
>      >     >>>>>> stuff
>      >
>      >     >>>>>> if the name is more generic.
>      >
>      >     >>>>>>
>      >
>      >     >>>>>>
>      >
>      >     >>>>>> -Matthias
>      >
>      >     >>>>>>
>      >
>      >     >>>>>>
>      >
>      >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>> I agree that the framework providing and managing the notion of
>      >
>      >     >>>>>>> stream
>      >
>      >     >>>>>>> time is valuable and not something we would want to delegate to
>      >
>      >     >> the
>      >
>      >     >>>>>>> tasks. I'm not entirely convinced that a separate callback
>      >
>      >     >> (option
>      >
>      >     >>>>>>> C)
>      >
>      >     >>>>>>> is that messy (it could just be a default method with an empty
>      >
>      >     >>>>>>> implementation), but if we wanted a single API to handle both
>      >
>      >     >>>>>>> cases,
>      >
>      >     >>>>>>> how about something like the following?
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>> enum Time {
>      >
>      >     >>>>>>>    STREAM,
>      >
>      >     >>>>>>>    CLOCK
>      >
>      >     >>>>>>> }
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>> Then on ProcessorContext:
>      >
>      >     >>>>>>> context.schedule(Time time, long interval)  // We could allow
>      >
>      >     >> this
>      >
>      >     >>>>>>> to
>      >
>      >     >>>>>>> be called once for each value of time to mix approaches.
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>> Then the Processor API becomes:
>      >
>      >     >>>>>>> punctuate(Time time) // time here denotes which schedule resulted
>      >
>      >     >>>>>>> in
>      >
>      >     >>>>>>> this call.
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>> Thoughts?
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> Thanks a lot for the KIP Michal,
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> I was thinking about the four options you proposed in more
>      >
>      >     >>>>>>>> details
>      >
>      >     >>>>>>>> and
>      >
>      >     >>>>>>>> this are my thoughts:
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> (A) You argue, that users can still "punctuate" on event-time
>      >
>      >     >> via
>      >
>      >     >>>>>>>> process(), but I am not sure if this is possible. Note, that
>      >
>      >     >>>>>>>> users
>      >
>      >     >>>>>>>> only
>      >
>      >     >>>>>>>> get record timestamps via context.timestamp(). Thus, users
>      >
>      >     >> would
>      >
>      >     >>>>>>>> need
>      >
>      >     >>>>>>>> to
>      >
>      >     >>>>>>>> track the time progress per partition (based on the partitions
>      >
>      >     >>>>>>>> they
>      >
>      >     >>>>>>>> obverse via context.partition(). (This alone puts a huge burden
>      >
>      >     >>>>>>>> on
>      >
>      >     >>>>>>>> the
>      >
>      >     >>>>>>>> user by itself.) However, users are not notified at startup
>      >
>      >     >> what
>      >
>      >     >>>>>>>> partitions are assigned, and user are not notified when
>      >
>      >     >>>>>>>> partitions
>      >
>      >     >>>>>>>> get
>      >
>      >     >>>>>>>> revoked. Because this information is not available, it's not
>      >
>      >     >>>>>>>> possible
>      >
>      >     >>>>>>>> to
>      >
>      >     >>>>>>>> "manually advance" stream-time, and thus event-time punctuation
>      >
>      >     >>>>>>>> within
>      >
>      >     >>>>>>>> process() seems not to be possible -- or do you see a way to
>      >
>      >     >> get
>      >
>      >     >>>>>>>> it
>      >
>      >     >>>>>>>> done? And even if, it might still be too clumsy to use.
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> (B) This does not allow to mix both approaches, thus limiting
>      >
>      >     >>>>>>>> what
>      >
>      >     >>>>>>>> users
>      >
>      >     >>>>>>>> can do.
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> (C) This should give all flexibility we need. However, just
>      >
>      >     >>>>>>>> adding
>      >
>      >     >>>>>>>> one
>      >
>      >     >>>>>>>> more method seems to be a solution that is too simple (cf my
>      >
>      >     >>>>>>>> comments
>      >
>      >     >>>>>>>> below).
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> (D) This might be hard to use. Also, I am not sure how a user
>      >
>      >     >>>>>>>> could
>      >
>      >     >>>>>>>> enable system-time and event-time punctuation in parallel.
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> Overall options (C) seems to be the most promising approach to
>      >
>      >     >>>>>>>> me.
>      >
>      >     >>>>>>>> Because I also favor a clean API, we might keep current
>      >
>      >     >>>>>>>> punctuate()
>      >
>      >     >>>>>>>> as-is, but deprecate it -- so we can remove it at some later
>      >
>      >     >>>>>>>> point
>      >
>      >     >>>>>>>> when
>      >
>      >     >>>>>>>> people use the "new punctuate API".
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> Couple of follow up questions:
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> - I am wondering, if we should have two callback methods or
>      >
>      >     >> just
>      >
>      >     >>>>>>>> one
>      >
>      >     >>>>>>>> (ie, a unified for system and event time punctuation or one for
>      >
>      >     >>>>>>>> each?).
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> - If we have one, how can the user figure out, which condition
>      >
>      >     >>>>>>>> did
>      >
>      >     >>>>>>>> trigger?
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> - How would the API look like, for registering different
>      >
>      >     >>>>>>>> punctuate
>      >
>      >     >>>>>>>> schedules? The "type" must be somehow defined?
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> - We might want to add "complex" schedules later on (like,
>      >
>      >     >>>>>>>> punctuate
>      >
>      >     >>>>>>>> on
>      >
>      >     >>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever
>      >
>      >     >>>>>>>> comes
>      >
>      >     >>>>>>>> first). I don't say we should add this right away, but we might
>      >
>      >     >>>>>>>> want
>      >
>      >     >>>>>>>> to
>      >
>      >     >>>>>>>> define the API in a way, that it allows extensions like this
>      >
>      >     >>>>>>>> later
>      >
>      >     >>>>>>>> on,
>      >
>      >     >>>>>>>> without redesigning the API (ie, the API should be designed
>      >
>      >     >>>>>>>> extensible)
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> - Did you ever consider count-based punctuation?
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> I understand, that you would like to solve a simple problem,
>      >
>      >     >> but
>      >
>      >     >>>>>>>> we
>      >
>      >     >>>>>>>> learned from the past, that just "adding some API" quickly
>      >
>      >     >> leads
>      >
>      >     >>>>>>>> to a
>      >
>      >     >>>>>>>> not very well defined API that needs time consuming clean up
>      >
>      >     >>>>>>>> later on
>      >
>      >     >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
>      >
>      >     >>>>>>>> punctuation
>      >
>      >     >>>>>>>> KIP
>      >
>      >     >>>>>>>> with this from the beginning on to avoid later painful
>      >
>      >     >> redesign.
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> -Matthias
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>>
>      >
>      >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>> Thanks Thomas,
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>> I'm also wary of changing the existing semantics of
>      >
>      >     >> punctuate,
>      >
>      >     >>>>>>>>> for
>      >
>      >     >>>>>>>>> backward compatibility reasons, although I like the
>      >
>      >     >> conceptual
>      >
>      >     >>>>>>>>> simplicity of that option.
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.
>      >
>      >     >> I
>      >
>      >     >>>>>>>>> added
>      >
>      >     >>>>>>>>> this to the KIP now as option (C).
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>> The TimestampExtractor mechanism is actually more flexible,
>      >
>      >     >> as
>      >
>      >     >>>>>>>>> it
>      >
>      >     >>>>>>>>> allows
>      >
>      >     >>>>>>>>> you to return any value, you're not limited to event time or
>      >
>      >     >>>>>>>>> system
>      >
>      >     >>>>>>>>> time
>      >
>      >     >>>>>>>>> (although I don't see an actual use case where you might need
>      >
>      >     >>>>>>>>> anything
>      >
>      >     >>>>>>>>> else then those two). Hence I also proposed the option to
>      >
>      >     >> allow
>      >
>      >     >>>>>>>>> users
>      >
>      >     >>>>>>>>> to, effectively, decide what "stream time" is for them given
>      >
>      >     >>>>>>>>> the
>      >
>      >     >>>>>>>>> presence or absence of messages, much like they can decide
>      >
>      >     >> what
>      >
>      >     >>>>>>>>> msg
>      >
>      >     >>>>>>>>> time
>      >
>      >     >>>>>>>>> means for them using the TimestampExtractor. What do you
>      >
>      >     >> think
>      >
>      >     >>>>>>>>> about
>      >
>      >     >>>>>>>>> that? This is probably most flexible but also most
>      >
>      >     >> complicated.
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>> All comments appreciated.
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>> Cheers,
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>> Michal
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>>
>      >
>      >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>      >
>      >     >>>>>>>>>>
>      >
>      >     >>>>>>>>>>
>      >
>      >     >>>>>>>>>> Although I fully agree we need a way to trigger periodic
>      >
>      >     >>>>>>>>>> processing
>      >
>      >     >>>>>>>>>> that is independent from whether and when messages arrive,
>      >
>      >     >>>>>>>>>> I'm
>      >
>      >     >>>>>>>>>> not sure
>      >
>      >     >>>>>>>>>> I like the idea of changing the existing semantics across
>      >
>      >     >> the
>      >
>      >     >>>>>>>>>> board.
>      >
>      >     >>>>>>>>>> What if we added an additional callback to Processor that
>      >
>      >     >> can
>      >
>      >     >>>>>>>>>> be
>      >
>      >     >>>>>>>>>> scheduled similarly to punctuate() but was always called at
>      >
>      >     >>>>>>>>>> fixed, wall
>      >
>      >     >>>>>>>>>> clock based intervals? This way you wouldn't have to give
>      >
>      >     >> up
>      >
>      >     >>>>>>>>>> the
>      >
>      >     >>>>>>>>>> notion
>      >
>      >     >>>>>>>>>> of stream time to be able to do periodic processing.
>      >
>      >     >>>>>>>>>>
>      >
>      >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>      >
>      >     >>>>>>>>>>>
>      >
>      >     >>>>>>>>>>>
>      >
>      >     >>>>>>>>>>> Hi all,
>      >
>      >     >>>>>>>>>>>
>      >
>      >     >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
>      >
>      >     >>>>>>>>>>> semantics
>      >
>      >     >>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP- <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>      >
>      >     > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>> 138%
>      >
>      >     >>>>>>>>>>> 3A+C
>      >
>      >     >>>>>>>>>>> hange+
>      >
>      >     >>>>>>>>>>> punctuate+semantics>
>      >
>      >     >>>>>>>>>>> .
>      >
>      >     >>>>>>>>>>>
>      >
>      >     >>>>>>>>>>> Appreciating there can be different views on system-time
>      >
>      >     >> vs
>      >
>      >     >>>>>>>>>>> event-
>      >
>      >     >>>>>>>>>>> time
>      >
>      >     >>>>>>>>>>> semantics for punctuation depending on use-case and the
>      >
>      >     >>>>>>>>>>> importance of
>      >
>      >     >>>>>>>>>>> backwards compatibility of any such change, I've left it
>      >
>      >     >>>>>>>>>>> quite
>      >
>      >     >>>>>>>>>>> open
>      >
>      >     >>>>>>>>>>> and
>      >
>      >     >>>>>>>>>>> hope to fill in more info as the discussion progresses.
>      >
>      >     >>>>>>>>>>>
>      >
>      >     >>>>>>>>>>> Thanks,
>      >
>      >     >>>>>>>>>>> Michal
>      >
>      >     >>>>>>> --
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>>     Tommy Becker
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>>     Senior Software Engineer
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>>     tivo.com
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>> ________________________________
>      >
>      >     >>>>>>>
>      >
>      >     >>>>>>> This email and any attachments may contain confidential and
>      >
>      >     >>>>>>> privileged material for the sole use of the intended recipient.
>      >
>      >     >> Any
>      >
>      >     >>>>>>> review, copying, or distribution of this email (or any
>      >
>      >     >> attachments)
>      >
>      >     >>>>>>> by others is prohibited. If you are not the intended recipient,
>      >
>      >     >>>>>>> please contact the sender immediately and permanently delete this
>      >
>      >     >>>>>>> email and any attachments. No employee or agent of TiVo Inc. is
>      >
>      >     >>>>>>> authorized to conclude any binding agreement on behalf of TiVo
>      >
>      >     >> Inc.
>      >
>      >     >>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a
>      >
>      >     >>>>>>> signed written agreement.
>      >
>      >     >>>>>>>
>      >
>      >     >>>>> --
>      >
>      >     >>>>>
>      >
>      >     >>>>>
>      >
>      >     >>>>>     Tommy Becker
>      >
>      >     >>>>>
>      >
>      >     >>>>>     Senior Software Engineer
>      >
>      >     >>>>>
>      >
>      >     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
>      >
>      >     >>>>>
>      >
>      >     >>>>>     tivo.com
>      >
>      >     >>>>>
>      >
>      >     >>>>>
>      >
>      >     >>>>> ________________________________
>      >
>      >     >>>>>
>      >
>      >     >>>>> This email and any attachments may contain confidential and
>      >
>      >     >> privileged
>      >
>      >     >>>>> material for the sole use of the intended recipient. Any review,
>      >
>      >     >>> copying,
>      >
>      >     >>>>> or distribution of this email (or any attachments) by others is
>      >
>      >     >>>> prohibited.
>      >
>      >     >>>>> If you are not the intended recipient, please contact the sender
>      >
>      >     >>>>> immediately and permanently delete this email and any attachments. No
>      >
>      >     >>>>> employee or agent of TiVo Inc. is authorized to conclude any binding
>      >
>      >     >>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with
>      >
>      >     >> TiVo
>      >
>      >     >>>>> Inc. may only be made by a signed written agreement.
>      >
>      >     >>>>>
>      >
>      >     >>>>
>      >
>      >     >>>
>      >
>      >     >>
>      >
>      >     >
>      >
>      >
>      >
>      >
>      >
>      >
>      >
>      >
>      >
>      > --
>      >
>      > <http://www.openbet.com/>
>      >
>      > *Michal Borowiecki*
>      >
>      > *Senior Software Engineer L4*
>      >
>      > *T: *
>      >
>      > +44 208 742 1600 <+44%2020%208742%201600>
>      >
>      > +44 203 249 8448 <+44%2020%203249%208448>
>      >
>      >
>      >
>      > *E: *
>      >
>      > michal.borowiecki@openbet.com
>      >
>      > *W: *
>      >
>      > www.openbet.com
>      >
>      > *OpenBet Ltd*
>      >
>      > Chiswick Park Building 9
>      >
>      > 566 Chiswick High Rd
>      >
>      > London
>      >
>      > W4 5XT
>      >
>      > UK
>      >
>      > <https://www.openbet.com/email_promo>
>      >
>      >
>      >
>      > This message is confidential and intended only for the addressee. If you
>      > have received this message in error, please immediately notify the
>      > postmaster@openbet.com and delete it from your system as well as any
>      > copies. The content of e-mails as well as traffic data may be monitored by
>      > OpenBet for employment and security purposes. To protect the environment
>      > please do not print this e-mail unless necessary. OpenBet Ltd. Registered
>      > Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
>      > United Kingdom. A company registered in England and Wales. Registered no.
>      > 3134634. VAT no. GB927523612
>      >
>      >
>      >
>      
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Arun Mathew <am...@yahoo-corp.jp>.
Thanks Ewen.

@Michal, @all, I have created a child page to start the Use Cases discussion [https://cwiki.apache.org/confluence/display/KAFKA/Punctuate+Use+Cases]. Please go through it and give your comments.

@Tianji, Sorry for the delay. I am trying to make the patch public.

--
Arun Mathew

On 4/8/17, 02:00, "Ewen Cheslack-Postava" <ew...@confluent.io> wrote:

    Arun,
    
    I've given you permission to edit the wiki. Let me know if you run into any
    issues.
    
    -Ewen
    
    On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <am...@yahoo-corp.jp> wrote:
    
    > Thanks Michal. I don’t have the access yet [arunmathew88]. Should I be
    > sending a separate mail for this?
    >
    > I thought one of the person following this thread would be able to give me
    > access.
    >
    >
    >
    > *From: *Michal Borowiecki <mi...@openbet.com>
    > *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
    > *Date: *Friday, April 7, 2017 at 17:16
    > *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
    > *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
    >
    >
    >
    > Hi Arun,
    >
    > I was thinking along the same lines as you, listing the use cases on the
    > wiki, but didn't find time to get around doing that yet.
    > Don't mind if you do it if you have access now.
    > I was thinking it would be nice if, once we have the use cases listed,
    > people could use likes to up-vote the use cases similar to what they're
    > working on.
    >
    > I should have a bit more time to action this in the next few days, but
    > happy for you to do it if you can beat me to it ;-)
    >
    > Cheers,
    > Michal
    >
    > On 07/04/17 04:39, Arun Mathew wrote:
    >
    > Sure, Thanks Matthias. My id is [arunmathew88].
    >
    >
    >
    > Of course. I was thinking of a subpage where people can collaborate.
    >
    >
    >
    > Will do as per Michael’s suggestion.
    >
    >
    >
    > Regards,
    >
    > Arun Mathew
    >
    >
    >
    > On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io> wrote:
    >
    >
    >
    >     Please share your Wiki-ID and a committer can give you write access.
    >
    >
    >
    >     Btw: as you did not initiate the KIP, you should not change the KIP
    >
    >     without the permission of the original author -- in this case Michael.
    >
    >
    >
    >     So you might also just share your thought over the mailing list and
    >
    >     Michael can update the KIP page. Or, as an alternative, just create a
    >
    >     subpage for the KIP page.
    >
    >
    >
    >     @Michael: WDYT?
    >
    >
    >
    >
    >
    >     -Matthias
    >
    >
    >
    >
    >
    >     On 4/6/17 8:05 PM, Arun Mathew wrote:
    >
    >     > Hi Jay,
    >
    >     >           Thanks for the advise, I would like to list down the use cases as
    >
    >     > per your suggestion. But it seems I don't have write permission to the
    >
    >     > Apache Kafka Confluent Space. Whom shall I request for it?
    >
    >     >
    >
    >     > Regarding your last question. We are using a patch in our production system
    >
    >     > which does exactly this.
    >
    >     > We window by the event time, but trigger punctuate in <punctuate interval>
    >
    >     > duration of system time, in the absence of an event crossing the punctuate
    >
    >     > event time.
    >
    >     >
    >
    >     > We are using Kafka Stream for our Audit Trail, where we need to output the
    >
    >     > event counts on each topic on each cluster aggregated over a 1 minute
    >
    >     > window. We have to use event time to be able to cross check the counts. But
    >
    >     > we need to trigger punctuate [aggregate event pushes] by system time in the
    >
    >     > absence of events. Otherwise the event counts for unexpired windows would
    >
    >     > be 0 which is bad.
    >
    >     >
    >
    >     > "Maybe a hybrid solution works: I window by event time but trigger results
    >
    >     > by system time for windows that have updated? Not really sure the details
    >
    >     > of making that work. Does that work? Are there concrete examples where you
    >
    >     > actually want the current behavior?"
    >
    >     >
    >
    >     > --
    >
    >     > With Regards,
    >
    >     >
    >
    >     > Arun Mathew
    >
    >     > Yahoo! JAPAN Corporation
    >
    >     >
    >
    >     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> <sk...@gmail.com> wrote:
    >
    >     >
    >
    >     >> Hi Jay,
    >
    >     >>
    >
    >     >> The hybrid solution is exactly what I expect and need for our use cases
    >
    >     >> when dealing with telecom data.
    >
    >     >>
    >
    >     >> Thanks
    >
    >     >> Tianji
    >
    >     >>
    >
    >     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> <ja...@confluent.io> wrote:
    >
    >     >>
    >
    >     >>> Hey guys,
    >
    >     >>>
    >
    >     >>> One thing I've always found super important for this kind of design work
    >
    >     >> is
    >
    >     >>> to do a really good job of cataloging the landscape of use cases and how
    >
    >     >>> prevalent each one is. By that I mean not just listing lots of uses, but
    >
    >     >>> also grouping them into categories that functionally need the same thing.
    >
    >     >>> In the absence of this it is very hard to reason about design proposals.
    >
    >     >>> From the proposals so far I think we have a lot of discussion around
    >
    >     >>> possible apis, but less around what the user needs for different use
    >
    >     >> cases
    >
    >     >>> and how they would implement that using the api.
    >
    >     >>>
    >
    >     >>> Here is an example:
    >
    >     >>> You aggregate click and impression data for a reddit like site. Every ten
    >
    >     >>> minutes you want to output a ranked list of the top 10 articles ranked by
    >
    >     >>> clicks/impressions for each geographical area. I want to be able run this
    >
    >     >>> in steady state as well as rerun to regenerate results (or catch up if it
    >
    >     >>> crashes).
    >
    >     >>>
    >
    >     >>> There are a couple of tricky things that seem to make this hard with
    >
    >     >> either
    >
    >     >>> of the options proposed:
    >
    >     >>> 1. If I emit this data using event time I have the problem described
    >
    >     >> where
    >
    >     >>> a geographical region with no new clicks or impressions will fail to
    >
    >     >> output
    >
    >     >>> results.
    >
    >     >>> 2. If I emit this data using system time I have the problem that when
    >
    >     >>> reprocessing data my window may not be ten minutes but 10 hours if my
    >
    >     >>> processing is very fast so it dramatically changes the output.
    >
    >     >>>
    >
    >     >>> Maybe a hybrid solution works: I window by event time but trigger results
    >
    >     >>> by system time for windows that have updated? Not really sure the details
    >
    >     >>> of making that work. Does that work? Are there concrete examples where
    >
    >     >> you
    >
    >     >>> actually want the current behavior?
    >
    >     >>>
    >
    >     >>> -Jay
    >
    >     >>>
    >
    >     >>>
    >
    >     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
    >
    >     >>> wrote:
    >
    >     >>>
    >
    >     >>>> Hi All,
    >
    >     >>>>
    >
    >     >>>> Thanks for the KIP. We were also in need of a mechanism to trigger
    >
    >     >>>> punctuate in the absence of events.
    >
    >     >>>>
    >
    >     >>>> As I described in [
    >
    >     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
    >
    >     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
    >
    >     >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
    >
    >     >>>> ],
    >
    >     >>>>
    >
    >     >>>>    - Our approached involved using the event time by default.
    >
    >     >>>>    - The method to check if there is any punctuate ready in the
    >
    >     >>>>    PunctuationQueue is triggered via the any event received by the
    >
    >     >> stream
    >
    >     >>>>    tread, or at the polling intervals in the absence of any events.
    >
    >     >>>>    - When we create Punctuate objects (which contains the next event
    >
    >     >> time
    >
    >     >>>>    for punctuation and interval), we also record the creation time
    >
    >     >>> (system
    >
    >     >>>>    time).
    >
    >     >>>>    - While checking for maturity of Punctuate Schedule by
    >
    >     >> mayBePunctuate
    >
    >     >>>>    method, we also check if the system clock has elapsed the punctuate
    >
    >     >>>>    interval since the schedule creation time.
    >
    >     >>>>    - In the absence of any event, or in the absence of any event for
    >
    >     >> one
    >
    >     >>>>    topic in the partition group assigned to the stream task, the system
    >
    >     >>>> time
    >
    >     >>>>    will elapse the interval and we trigger a punctuate using the
    >
    >     >> expected
    >
    >     >>>>    punctuation event time.
    >
    >     >>>>    - we then create the next punctuation schedule as punctuation event
    >
    >     >>> time
    >
    >     >>>>    + punctuation interval, [again recording the system time of creation
    >
    >     >>> of
    >
    >     >>>> the
    >
    >     >>>>    schedule].
    >
    >     >>>>
    >
    >     >>>> We call this a Hybrid Punctuate. Of course, this approach has pros and
    >
    >     >>>> cons.
    >
    >     >>>> Pros
    >
    >     >>>>
    >
    >     >>>>    - Punctuates will happen in <punctuate interval> time duration at
    >
    >     >> max
    >
    >     >>> in
    >
    >     >>>>    terms of system time.
    >
    >     >>>>    - The semantics as a whole continues to revolve around event time.
    >
    >     >>>>    - We can use the old data [old timestamps] to rerun any experiments
    >
    >     >> or
    >
    >     >>>>    tests.
    >
    >     >>>>
    >
    >     >>>> Cons
    >
    >     >>>>
    >
    >     >>>>    - In case the  <punctuate interval> is not a time duration [say
    >
    >     >>> logical
    >
    >     >>>>    time/event count], then the approach might not be meaningful.
    >
    >     >>>>    - In case there is a case where we have to wait for an actual event
    >
    >     >>> from
    >
    >     >>>>    a low event rate partition in the partition group, this approach
    >
    >     >> will
    >
    >     >>>> jump
    >
    >     >>>>    the gun.
    >
    >     >>>>    - in case the event processing cannot catch up with the event rate
    >
    >     >> and
    >
    >     >>>>    the expected timestamp events gets queued for long time, this
    >
    >     >> approach
    >
    >     >>>>    might jump the gun.
    >
    >     >>>>
    >
    >     >>>> I believe the above approach and discussion goes close to the approach
    >
    >     >> A.
    >
    >     >>>>
    >
    >     >>>> -----------
    >
    >     >>>>
    >
    >     >>>> I like the idea of having an even count based punctuate.
    >
    >     >>>>
    >
    >     >>>> -----------
    >
    >     >>>>
    >
    >     >>>> I agree with the discussion around approach C, that we should provide
    >
    >     >> the
    >
    >     >>>> user with the option to choose system time or event time based
    >
    >     >>> punctuates.
    >
    >     >>>> But I believe that the user predominantly wants to use event time while
    >
    >     >>> not
    >
    >     >>>> missing out on regular punctuates due to event delays or event
    >
    >     >> absences.
    >
    >     >>>> Hence a complex punctuate option as Matthias mentioned (quoted below)
    >
    >     >>> would
    >
    >     >>>> be most apt.
    >
    >     >>>>
    >
    >     >>>> "- We might want to add "complex" schedules later on (like, punctuate
    >
    >     >> on
    >
    >     >>>> every 10 seconds event-time or 60 seconds system-time whatever comes
    >
    >     >>>> first)."
    >
    >     >>>>
    >
    >     >>>> -----------
    >
    >     >>>>
    >
    >     >>>> I think I read somewhere that Kafka Streams started with System Time as
    >
    >     >>> the
    >
    >     >>>> punctuation standard, but was later changed to Event Time. I guess
    >
    >     >> there
    >
    >     >>>> would be some good reason behind it. As Kafka Streams want to evolve
    >
    >     >> more
    >
    >     >>>> on the Stream Processing front, I believe the emphasis on event time
    >
    >     >>> would
    >
    >     >>>> remain quite strong.
    >
    >     >>>>
    >
    >     >>>>
    >
    >     >>>> With Regards,
    >
    >     >>>>
    >
    >     >>>> Arun Mathew
    >
    >     >>>> Yahoo! JAPAN Corporation, Tokyo
    >
    >     >>>>
    >
    >     >>>>
    >
    >     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com> <to...@tivo.com>
    >
    >     >> wrote:
    >
    >     >>>>
    >
    >     >>>>> Yeah I like PuncutationType much better; I just threw Time out there
    >
    >     >>>>> more as a strawman than an actual suggestion ;) I still think it's
    >
    >     >>>>> worth considering what this buys us over an additional callback. I
    >
    >     >>>>> foresee a number of punctuate implementations following this pattern:
    >
    >     >>>>>
    >
    >     >>>>> public void punctuate(PunctuationType type) {
    >
    >     >>>>>     switch (type) {
    >
    >     >>>>>         case EVENT_TIME:
    >
    >     >>>>>             methodA();
    >
    >     >>>>>             break;
    >
    >     >>>>>         case SYSTEM_TIME:
    >
    >     >>>>>             methodB();
    >
    >     >>>>>             break;
    >
    >     >>>>>     }
    >
    >     >>>>> }
    >
    >     >>>>>
    >
    >     >>>>> I guess one advantage of this approach is we could add additional
    >
    >     >>>>> punctuation types later in a backwards compatible way (like event
    >
    >     >> count
    >
    >     >>>>> as you mentioned).
    >
    >     >>>>>
    >
    >     >>>>> -Tommy
    >
    >     >>>>>
    >
    >     >>>>>
    >
    >     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
    >
    >     >>>>>> That sounds promising.
    >
    >     >>>>>>
    >
    >     >>>>>> I am just wondering if `Time` is the best name. Maybe we want to
    >
    >     >> add
    >
    >     >>>>>> other non-time based punctuations at some point later. I would
    >
    >     >>>>>> suggest
    >
    >     >>>>>>
    >
    >     >>>>>> enum PunctuationType {
    >
    >     >>>>>>   EVENT_TIME,
    >
    >     >>>>>>   SYSTEM_TIME,
    >
    >     >>>>>> }
    >
    >     >>>>>>
    >
    >     >>>>>> or similar. Just to keep the door open -- it's easier to add new
    >
    >     >>>>>> stuff
    >
    >     >>>>>> if the name is more generic.
    >
    >     >>>>>>
    >
    >     >>>>>>
    >
    >     >>>>>> -Matthias
    >
    >     >>>>>>
    >
    >     >>>>>>
    >
    >     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
    >
    >     >>>>>>>
    >
    >     >>>>>>> I agree that the framework providing and managing the notion of
    >
    >     >>>>>>> stream
    >
    >     >>>>>>> time is valuable and not something we would want to delegate to
    >
    >     >> the
    >
    >     >>>>>>> tasks. I'm not entirely convinced that a separate callback
    >
    >     >> (option
    >
    >     >>>>>>> C)
    >
    >     >>>>>>> is that messy (it could just be a default method with an empty
    >
    >     >>>>>>> implementation), but if we wanted a single API to handle both
    >
    >     >>>>>>> cases,
    >
    >     >>>>>>> how about something like the following?
    >
    >     >>>>>>>
    >
    >     >>>>>>> enum Time {
    >
    >     >>>>>>>    STREAM,
    >
    >     >>>>>>>    CLOCK
    >
    >     >>>>>>> }
    >
    >     >>>>>>>
    >
    >     >>>>>>> Then on ProcessorContext:
    >
    >     >>>>>>> context.schedule(Time time, long interval)  // We could allow
    >
    >     >> this
    >
    >     >>>>>>> to
    >
    >     >>>>>>> be called once for each value of time to mix approaches.
    >
    >     >>>>>>>
    >
    >     >>>>>>> Then the Processor API becomes:
    >
    >     >>>>>>> punctuate(Time time) // time here denotes which schedule resulted
    >
    >     >>>>>>> in
    >
    >     >>>>>>> this call.
    >
    >     >>>>>>>
    >
    >     >>>>>>> Thoughts?
    >
    >     >>>>>>>
    >
    >     >>>>>>>
    >
    >     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> Thanks a lot for the KIP Michal,
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> I was thinking about the four options you proposed in more
    >
    >     >>>>>>>> details
    >
    >     >>>>>>>> and
    >
    >     >>>>>>>> this are my thoughts:
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> (A) You argue, that users can still "punctuate" on event-time
    >
    >     >> via
    >
    >     >>>>>>>> process(), but I am not sure if this is possible. Note, that
    >
    >     >>>>>>>> users
    >
    >     >>>>>>>> only
    >
    >     >>>>>>>> get record timestamps via context.timestamp(). Thus, users
    >
    >     >> would
    >
    >     >>>>>>>> need
    >
    >     >>>>>>>> to
    >
    >     >>>>>>>> track the time progress per partition (based on the partitions
    >
    >     >>>>>>>> they
    >
    >     >>>>>>>> obverse via context.partition(). (This alone puts a huge burden
    >
    >     >>>>>>>> on
    >
    >     >>>>>>>> the
    >
    >     >>>>>>>> user by itself.) However, users are not notified at startup
    >
    >     >> what
    >
    >     >>>>>>>> partitions are assigned, and user are not notified when
    >
    >     >>>>>>>> partitions
    >
    >     >>>>>>>> get
    >
    >     >>>>>>>> revoked. Because this information is not available, it's not
    >
    >     >>>>>>>> possible
    >
    >     >>>>>>>> to
    >
    >     >>>>>>>> "manually advance" stream-time, and thus event-time punctuation
    >
    >     >>>>>>>> within
    >
    >     >>>>>>>> process() seems not to be possible -- or do you see a way to
    >
    >     >> get
    >
    >     >>>>>>>> it
    >
    >     >>>>>>>> done? And even if, it might still be too clumsy to use.
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> (B) This does not allow to mix both approaches, thus limiting
    >
    >     >>>>>>>> what
    >
    >     >>>>>>>> users
    >
    >     >>>>>>>> can do.
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> (C) This should give all flexibility we need. However, just
    >
    >     >>>>>>>> adding
    >
    >     >>>>>>>> one
    >
    >     >>>>>>>> more method seems to be a solution that is too simple (cf my
    >
    >     >>>>>>>> comments
    >
    >     >>>>>>>> below).
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> (D) This might be hard to use. Also, I am not sure how a user
    >
    >     >>>>>>>> could
    >
    >     >>>>>>>> enable system-time and event-time punctuation in parallel.
    >
    >     >>>>>>>>
    >
    >     >>>>>>>>
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> Overall options (C) seems to be the most promising approach to
    >
    >     >>>>>>>> me.
    >
    >     >>>>>>>> Because I also favor a clean API, we might keep current
    >
    >     >>>>>>>> punctuate()
    >
    >     >>>>>>>> as-is, but deprecate it -- so we can remove it at some later
    >
    >     >>>>>>>> point
    >
    >     >>>>>>>> when
    >
    >     >>>>>>>> people use the "new punctuate API".
    >
    >     >>>>>>>>
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> Couple of follow up questions:
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> - I am wondering, if we should have two callback methods or
    >
    >     >> just
    >
    >     >>>>>>>> one
    >
    >     >>>>>>>> (ie, a unified for system and event time punctuation or one for
    >
    >     >>>>>>>> each?).
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> - If we have one, how can the user figure out, which condition
    >
    >     >>>>>>>> did
    >
    >     >>>>>>>> trigger?
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> - How would the API look like, for registering different
    >
    >     >>>>>>>> punctuate
    >
    >     >>>>>>>> schedules? The "type" must be somehow defined?
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> - We might want to add "complex" schedules later on (like,
    >
    >     >>>>>>>> punctuate
    >
    >     >>>>>>>> on
    >
    >     >>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever
    >
    >     >>>>>>>> comes
    >
    >     >>>>>>>> first). I don't say we should add this right away, but we might
    >
    >     >>>>>>>> want
    >
    >     >>>>>>>> to
    >
    >     >>>>>>>> define the API in a way, that it allows extensions like this
    >
    >     >>>>>>>> later
    >
    >     >>>>>>>> on,
    >
    >     >>>>>>>> without redesigning the API (ie, the API should be designed
    >
    >     >>>>>>>> extensible)
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> - Did you ever consider count-based punctuation?
    >
    >     >>>>>>>>
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> I understand, that you would like to solve a simple problem,
    >
    >     >> but
    >
    >     >>>>>>>> we
    >
    >     >>>>>>>> learned from the past, that just "adding some API" quickly
    >
    >     >> leads
    >
    >     >>>>>>>> to a
    >
    >     >>>>>>>> not very well defined API that needs time consuming clean up
    >
    >     >>>>>>>> later on
    >
    >     >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
    >
    >     >>>>>>>> punctuation
    >
    >     >>>>>>>> KIP
    >
    >     >>>>>>>> with this from the beginning on to avoid later painful
    >
    >     >> redesign.
    >
    >     >>>>>>>>
    >
    >     >>>>>>>>
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> -Matthias
    >
    >     >>>>>>>>
    >
    >     >>>>>>>>
    >
    >     >>>>>>>>
    >
    >     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>> Thanks Thomas,
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>> I'm also wary of changing the existing semantics of
    >
    >     >> punctuate,
    >
    >     >>>>>>>>> for
    >
    >     >>>>>>>>> backward compatibility reasons, although I like the
    >
    >     >> conceptual
    >
    >     >>>>>>>>> simplicity of that option.
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.
    >
    >     >> I
    >
    >     >>>>>>>>> added
    >
    >     >>>>>>>>> this to the KIP now as option (C).
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>> The TimestampExtractor mechanism is actually more flexible,
    >
    >     >> as
    >
    >     >>>>>>>>> it
    >
    >     >>>>>>>>> allows
    >
    >     >>>>>>>>> you to return any value, you're not limited to event time or
    >
    >     >>>>>>>>> system
    >
    >     >>>>>>>>> time
    >
    >     >>>>>>>>> (although I don't see an actual use case where you might need
    >
    >     >>>>>>>>> anything
    >
    >     >>>>>>>>> else then those two). Hence I also proposed the option to
    >
    >     >> allow
    >
    >     >>>>>>>>> users
    >
    >     >>>>>>>>> to, effectively, decide what "stream time" is for them given
    >
    >     >>>>>>>>> the
    >
    >     >>>>>>>>> presence or absence of messages, much like they can decide
    >
    >     >> what
    >
    >     >>>>>>>>> msg
    >
    >     >>>>>>>>> time
    >
    >     >>>>>>>>> means for them using the TimestampExtractor. What do you
    >
    >     >> think
    >
    >     >>>>>>>>> about
    >
    >     >>>>>>>>> that? This is probably most flexible but also most
    >
    >     >> complicated.
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>> All comments appreciated.
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>> Cheers,
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>> Michal
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>>
    >
    >     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
    >
    >     >>>>>>>>>>
    >
    >     >>>>>>>>>>
    >
    >     >>>>>>>>>> Although I fully agree we need a way to trigger periodic
    >
    >     >>>>>>>>>> processing
    >
    >     >>>>>>>>>> that is independent from whether and when messages arrive,
    >
    >     >>>>>>>>>> I'm
    >
    >     >>>>>>>>>> not sure
    >
    >     >>>>>>>>>> I like the idea of changing the existing semantics across
    >
    >     >> the
    >
    >     >>>>>>>>>> board.
    >
    >     >>>>>>>>>> What if we added an additional callback to Processor that
    >
    >     >> can
    >
    >     >>>>>>>>>> be
    >
    >     >>>>>>>>>> scheduled similarly to punctuate() but was always called at
    >
    >     >>>>>>>>>> fixed, wall
    >
    >     >>>>>>>>>> clock based intervals? This way you wouldn't have to give
    >
    >     >> up
    >
    >     >>>>>>>>>> the
    >
    >     >>>>>>>>>> notion
    >
    >     >>>>>>>>>> of stream time to be able to do periodic processing.
    >
    >     >>>>>>>>>>
    >
    >     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
    >
    >     >>>>>>>>>>>
    >
    >     >>>>>>>>>>>
    >
    >     >>>>>>>>>>> Hi all,
    >
    >     >>>>>>>>>>>
    >
    >     >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
    >
    >     >>>>>>>>>>> semantics
    >
    >     >>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP- <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
    >
    >     > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>> 138%
    >
    >     >>>>>>>>>>> 3A+C
    >
    >     >>>>>>>>>>> hange+
    >
    >     >>>>>>>>>>> punctuate+semantics>
    >
    >     >>>>>>>>>>> .
    >
    >     >>>>>>>>>>>
    >
    >     >>>>>>>>>>> Appreciating there can be different views on system-time
    >
    >     >> vs
    >
    >     >>>>>>>>>>> event-
    >
    >     >>>>>>>>>>> time
    >
    >     >>>>>>>>>>> semantics for punctuation depending on use-case and the
    >
    >     >>>>>>>>>>> importance of
    >
    >     >>>>>>>>>>> backwards compatibility of any such change, I've left it
    >
    >     >>>>>>>>>>> quite
    >
    >     >>>>>>>>>>> open
    >
    >     >>>>>>>>>>> and
    >
    >     >>>>>>>>>>> hope to fill in more info as the discussion progresses.
    >
    >     >>>>>>>>>>>
    >
    >     >>>>>>>>>>> Thanks,
    >
    >     >>>>>>>>>>> Michal
    >
    >     >>>>>>> --
    >
    >     >>>>>>>
    >
    >     >>>>>>>
    >
    >     >>>>>>>     Tommy Becker
    >
    >     >>>>>>>
    >
    >     >>>>>>>     Senior Software Engineer
    >
    >     >>>>>>>
    >
    >     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
    >
    >     >>>>>>>
    >
    >     >>>>>>>     tivo.com
    >
    >     >>>>>>>
    >
    >     >>>>>>>
    >
    >     >>>>>>> ________________________________
    >
    >     >>>>>>>
    >
    >     >>>>>>> This email and any attachments may contain confidential and
    >
    >     >>>>>>> privileged material for the sole use of the intended recipient.
    >
    >     >> Any
    >
    >     >>>>>>> review, copying, or distribution of this email (or any
    >
    >     >> attachments)
    >
    >     >>>>>>> by others is prohibited. If you are not the intended recipient,
    >
    >     >>>>>>> please contact the sender immediately and permanently delete this
    >
    >     >>>>>>> email and any attachments. No employee or agent of TiVo Inc. is
    >
    >     >>>>>>> authorized to conclude any binding agreement on behalf of TiVo
    >
    >     >> Inc.
    >
    >     >>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a
    >
    >     >>>>>>> signed written agreement.
    >
    >     >>>>>>>
    >
    >     >>>>> --
    >
    >     >>>>>
    >
    >     >>>>>
    >
    >     >>>>>     Tommy Becker
    >
    >     >>>>>
    >
    >     >>>>>     Senior Software Engineer
    >
    >     >>>>>
    >
    >     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
    >
    >     >>>>>
    >
    >     >>>>>     tivo.com
    >
    >     >>>>>
    >
    >     >>>>>
    >
    >     >>>>> ________________________________
    >
    >     >>>>>
    >
    >     >>>>> This email and any attachments may contain confidential and
    >
    >     >> privileged
    >
    >     >>>>> material for the sole use of the intended recipient. Any review,
    >
    >     >>> copying,
    >
    >     >>>>> or distribution of this email (or any attachments) by others is
    >
    >     >>>> prohibited.
    >
    >     >>>>> If you are not the intended recipient, please contact the sender
    >
    >     >>>>> immediately and permanently delete this email and any attachments. No
    >
    >     >>>>> employee or agent of TiVo Inc. is authorized to conclude any binding
    >
    >     >>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with
    >
    >     >> TiVo
    >
    >     >>>>> Inc. may only be made by a signed written agreement.
    >
    >     >>>>>
    >
    >     >>>>
    >
    >     >>>
    >
    >     >>
    >
    >     >
    >
    >
    >
    >
    >
    >
    >
    >
    >
    > --
    >
    > <http://www.openbet.com/>
    >
    > *Michal Borowiecki*
    >
    > *Senior Software Engineer L4*
    >
    > *T: *
    >
    > +44 208 742 1600 <+44%2020%208742%201600>
    >
    > +44 203 249 8448 <+44%2020%203249%208448>
    >
    >
    >
    > *E: *
    >
    > michal.borowiecki@openbet.com
    >
    > *W: *
    >
    > www.openbet.com
    >
    > *OpenBet Ltd*
    >
    > Chiswick Park Building 9
    >
    > 566 Chiswick High Rd
    >
    > London
    >
    > W4 5XT
    >
    > UK
    >
    > <https://www.openbet.com/email_promo>
    >
    >
    >
    > This message is confidential and intended only for the addressee. If you
    > have received this message in error, please immediately notify the
    > postmaster@openbet.com and delete it from your system as well as any
    > copies. The content of e-mails as well as traffic data may be monitored by
    > OpenBet for employment and security purposes. To protect the environment
    > please do not print this e-mail unless necessary. OpenBet Ltd. Registered
    > Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
    > United Kingdom. A company registered in England and Wales. Registered no.
    > 3134634. VAT no. GB927523612
    >
    >
    >
    


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
Arun,

I've given you permission to edit the wiki. Let me know if you run into any
issues.

-Ewen

On Fri, Apr 7, 2017 at 1:21 AM, Arun Mathew <am...@yahoo-corp.jp> wrote:

> Thanks Michal. I don’t have the access yet [arunmathew88]. Should I be
> sending a separate mail for this?
>
> I thought one of the person following this thread would be able to give me
> access.
>
>
>
> *From: *Michal Borowiecki <mi...@openbet.com>
> *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> *Date: *Friday, April 7, 2017 at 17:16
> *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
>
>
>
> Hi Arun,
>
> I was thinking along the same lines as you, listing the use cases on the
> wiki, but didn't find time to get around doing that yet.
> Don't mind if you do it if you have access now.
> I was thinking it would be nice if, once we have the use cases listed,
> people could use likes to up-vote the use cases similar to what they're
> working on.
>
> I should have a bit more time to action this in the next few days, but
> happy for you to do it if you can beat me to it ;-)
>
> Cheers,
> Michal
>
> On 07/04/17 04:39, Arun Mathew wrote:
>
> Sure, Thanks Matthias. My id is [arunmathew88].
>
>
>
> Of course. I was thinking of a subpage where people can collaborate.
>
>
>
> Will do as per Michael’s suggestion.
>
>
>
> Regards,
>
> Arun Mathew
>
>
>
> On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io> wrote:
>
>
>
>     Please share your Wiki-ID and a committer can give you write access.
>
>
>
>     Btw: as you did not initiate the KIP, you should not change the KIP
>
>     without the permission of the original author -- in this case Michael.
>
>
>
>     So you might also just share your thought over the mailing list and
>
>     Michael can update the KIP page. Or, as an alternative, just create a
>
>     subpage for the KIP page.
>
>
>
>     @Michael: WDYT?
>
>
>
>
>
>     -Matthias
>
>
>
>
>
>     On 4/6/17 8:05 PM, Arun Mathew wrote:
>
>     > Hi Jay,
>
>     >           Thanks for the advise, I would like to list down the use cases as
>
>     > per your suggestion. But it seems I don't have write permission to the
>
>     > Apache Kafka Confluent Space. Whom shall I request for it?
>
>     >
>
>     > Regarding your last question. We are using a patch in our production system
>
>     > which does exactly this.
>
>     > We window by the event time, but trigger punctuate in <punctuate interval>
>
>     > duration of system time, in the absence of an event crossing the punctuate
>
>     > event time.
>
>     >
>
>     > We are using Kafka Stream for our Audit Trail, where we need to output the
>
>     > event counts on each topic on each cluster aggregated over a 1 minute
>
>     > window. We have to use event time to be able to cross check the counts. But
>
>     > we need to trigger punctuate [aggregate event pushes] by system time in the
>
>     > absence of events. Otherwise the event counts for unexpired windows would
>
>     > be 0 which is bad.
>
>     >
>
>     > "Maybe a hybrid solution works: I window by event time but trigger results
>
>     > by system time for windows that have updated? Not really sure the details
>
>     > of making that work. Does that work? Are there concrete examples where you
>
>     > actually want the current behavior?"
>
>     >
>
>     > --
>
>     > With Regards,
>
>     >
>
>     > Arun Mathew
>
>     > Yahoo! JAPAN Corporation
>
>     >
>
>     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> <sk...@gmail.com> wrote:
>
>     >
>
>     >> Hi Jay,
>
>     >>
>
>     >> The hybrid solution is exactly what I expect and need for our use cases
>
>     >> when dealing with telecom data.
>
>     >>
>
>     >> Thanks
>
>     >> Tianji
>
>     >>
>
>     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> <ja...@confluent.io> wrote:
>
>     >>
>
>     >>> Hey guys,
>
>     >>>
>
>     >>> One thing I've always found super important for this kind of design work
>
>     >> is
>
>     >>> to do a really good job of cataloging the landscape of use cases and how
>
>     >>> prevalent each one is. By that I mean not just listing lots of uses, but
>
>     >>> also grouping them into categories that functionally need the same thing.
>
>     >>> In the absence of this it is very hard to reason about design proposals.
>
>     >>> From the proposals so far I think we have a lot of discussion around
>
>     >>> possible apis, but less around what the user needs for different use
>
>     >> cases
>
>     >>> and how they would implement that using the api.
>
>     >>>
>
>     >>> Here is an example:
>
>     >>> You aggregate click and impression data for a reddit like site. Every ten
>
>     >>> minutes you want to output a ranked list of the top 10 articles ranked by
>
>     >>> clicks/impressions for each geographical area. I want to be able run this
>
>     >>> in steady state as well as rerun to regenerate results (or catch up if it
>
>     >>> crashes).
>
>     >>>
>
>     >>> There are a couple of tricky things that seem to make this hard with
>
>     >> either
>
>     >>> of the options proposed:
>
>     >>> 1. If I emit this data using event time I have the problem described
>
>     >> where
>
>     >>> a geographical region with no new clicks or impressions will fail to
>
>     >> output
>
>     >>> results.
>
>     >>> 2. If I emit this data using system time I have the problem that when
>
>     >>> reprocessing data my window may not be ten minutes but 10 hours if my
>
>     >>> processing is very fast so it dramatically changes the output.
>
>     >>>
>
>     >>> Maybe a hybrid solution works: I window by event time but trigger results
>
>     >>> by system time for windows that have updated? Not really sure the details
>
>     >>> of making that work. Does that work? Are there concrete examples where
>
>     >> you
>
>     >>> actually want the current behavior?
>
>     >>>
>
>     >>> -Jay
>
>     >>>
>
>     >>>
>
>     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>
>     >>> wrote:
>
>     >>>
>
>     >>>> Hi All,
>
>     >>>>
>
>     >>>> Thanks for the KIP. We were also in need of a mechanism to trigger
>
>     >>>> punctuate in the absence of events.
>
>     >>>>
>
>     >>>> As I described in [
>
>     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>
>     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>
>     >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
>
>     >>>> ],
>
>     >>>>
>
>     >>>>    - Our approached involved using the event time by default.
>
>     >>>>    - The method to check if there is any punctuate ready in the
>
>     >>>>    PunctuationQueue is triggered via the any event received by the
>
>     >> stream
>
>     >>>>    tread, or at the polling intervals in the absence of any events.
>
>     >>>>    - When we create Punctuate objects (which contains the next event
>
>     >> time
>
>     >>>>    for punctuation and interval), we also record the creation time
>
>     >>> (system
>
>     >>>>    time).
>
>     >>>>    - While checking for maturity of Punctuate Schedule by
>
>     >> mayBePunctuate
>
>     >>>>    method, we also check if the system clock has elapsed the punctuate
>
>     >>>>    interval since the schedule creation time.
>
>     >>>>    - In the absence of any event, or in the absence of any event for
>
>     >> one
>
>     >>>>    topic in the partition group assigned to the stream task, the system
>
>     >>>> time
>
>     >>>>    will elapse the interval and we trigger a punctuate using the
>
>     >> expected
>
>     >>>>    punctuation event time.
>
>     >>>>    - we then create the next punctuation schedule as punctuation event
>
>     >>> time
>
>     >>>>    + punctuation interval, [again recording the system time of creation
>
>     >>> of
>
>     >>>> the
>
>     >>>>    schedule].
>
>     >>>>
>
>     >>>> We call this a Hybrid Punctuate. Of course, this approach has pros and
>
>     >>>> cons.
>
>     >>>> Pros
>
>     >>>>
>
>     >>>>    - Punctuates will happen in <punctuate interval> time duration at
>
>     >> max
>
>     >>> in
>
>     >>>>    terms of system time.
>
>     >>>>    - The semantics as a whole continues to revolve around event time.
>
>     >>>>    - We can use the old data [old timestamps] to rerun any experiments
>
>     >> or
>
>     >>>>    tests.
>
>     >>>>
>
>     >>>> Cons
>
>     >>>>
>
>     >>>>    - In case the  <punctuate interval> is not a time duration [say
>
>     >>> logical
>
>     >>>>    time/event count], then the approach might not be meaningful.
>
>     >>>>    - In case there is a case where we have to wait for an actual event
>
>     >>> from
>
>     >>>>    a low event rate partition in the partition group, this approach
>
>     >> will
>
>     >>>> jump
>
>     >>>>    the gun.
>
>     >>>>    - in case the event processing cannot catch up with the event rate
>
>     >> and
>
>     >>>>    the expected timestamp events gets queued for long time, this
>
>     >> approach
>
>     >>>>    might jump the gun.
>
>     >>>>
>
>     >>>> I believe the above approach and discussion goes close to the approach
>
>     >> A.
>
>     >>>>
>
>     >>>> -----------
>
>     >>>>
>
>     >>>> I like the idea of having an even count based punctuate.
>
>     >>>>
>
>     >>>> -----------
>
>     >>>>
>
>     >>>> I agree with the discussion around approach C, that we should provide
>
>     >> the
>
>     >>>> user with the option to choose system time or event time based
>
>     >>> punctuates.
>
>     >>>> But I believe that the user predominantly wants to use event time while
>
>     >>> not
>
>     >>>> missing out on regular punctuates due to event delays or event
>
>     >> absences.
>
>     >>>> Hence a complex punctuate option as Matthias mentioned (quoted below)
>
>     >>> would
>
>     >>>> be most apt.
>
>     >>>>
>
>     >>>> "- We might want to add "complex" schedules later on (like, punctuate
>
>     >> on
>
>     >>>> every 10 seconds event-time or 60 seconds system-time whatever comes
>
>     >>>> first)."
>
>     >>>>
>
>     >>>> -----------
>
>     >>>>
>
>     >>>> I think I read somewhere that Kafka Streams started with System Time as
>
>     >>> the
>
>     >>>> punctuation standard, but was later changed to Event Time. I guess
>
>     >> there
>
>     >>>> would be some good reason behind it. As Kafka Streams want to evolve
>
>     >> more
>
>     >>>> on the Stream Processing front, I believe the emphasis on event time
>
>     >>> would
>
>     >>>> remain quite strong.
>
>     >>>>
>
>     >>>>
>
>     >>>> With Regards,
>
>     >>>>
>
>     >>>> Arun Mathew
>
>     >>>> Yahoo! JAPAN Corporation, Tokyo
>
>     >>>>
>
>     >>>>
>
>     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com> <to...@tivo.com>
>
>     >> wrote:
>
>     >>>>
>
>     >>>>> Yeah I like PuncutationType much better; I just threw Time out there
>
>     >>>>> more as a strawman than an actual suggestion ;) I still think it's
>
>     >>>>> worth considering what this buys us over an additional callback. I
>
>     >>>>> foresee a number of punctuate implementations following this pattern:
>
>     >>>>>
>
>     >>>>> public void punctuate(PunctuationType type) {
>
>     >>>>>     switch (type) {
>
>     >>>>>         case EVENT_TIME:
>
>     >>>>>             methodA();
>
>     >>>>>             break;
>
>     >>>>>         case SYSTEM_TIME:
>
>     >>>>>             methodB();
>
>     >>>>>             break;
>
>     >>>>>     }
>
>     >>>>> }
>
>     >>>>>
>
>     >>>>> I guess one advantage of this approach is we could add additional
>
>     >>>>> punctuation types later in a backwards compatible way (like event
>
>     >> count
>
>     >>>>> as you mentioned).
>
>     >>>>>
>
>     >>>>> -Tommy
>
>     >>>>>
>
>     >>>>>
>
>     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
>
>     >>>>>> That sounds promising.
>
>     >>>>>>
>
>     >>>>>> I am just wondering if `Time` is the best name. Maybe we want to
>
>     >> add
>
>     >>>>>> other non-time based punctuations at some point later. I would
>
>     >>>>>> suggest
>
>     >>>>>>
>
>     >>>>>> enum PunctuationType {
>
>     >>>>>>   EVENT_TIME,
>
>     >>>>>>   SYSTEM_TIME,
>
>     >>>>>> }
>
>     >>>>>>
>
>     >>>>>> or similar. Just to keep the door open -- it's easier to add new
>
>     >>>>>> stuff
>
>     >>>>>> if the name is more generic.
>
>     >>>>>>
>
>     >>>>>>
>
>     >>>>>> -Matthias
>
>     >>>>>>
>
>     >>>>>>
>
>     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>
>     >>>>>>>
>
>     >>>>>>> I agree that the framework providing and managing the notion of
>
>     >>>>>>> stream
>
>     >>>>>>> time is valuable and not something we would want to delegate to
>
>     >> the
>
>     >>>>>>> tasks. I'm not entirely convinced that a separate callback
>
>     >> (option
>
>     >>>>>>> C)
>
>     >>>>>>> is that messy (it could just be a default method with an empty
>
>     >>>>>>> implementation), but if we wanted a single API to handle both
>
>     >>>>>>> cases,
>
>     >>>>>>> how about something like the following?
>
>     >>>>>>>
>
>     >>>>>>> enum Time {
>
>     >>>>>>>    STREAM,
>
>     >>>>>>>    CLOCK
>
>     >>>>>>> }
>
>     >>>>>>>
>
>     >>>>>>> Then on ProcessorContext:
>
>     >>>>>>> context.schedule(Time time, long interval)  // We could allow
>
>     >> this
>
>     >>>>>>> to
>
>     >>>>>>> be called once for each value of time to mix approaches.
>
>     >>>>>>>
>
>     >>>>>>> Then the Processor API becomes:
>
>     >>>>>>> punctuate(Time time) // time here denotes which schedule resulted
>
>     >>>>>>> in
>
>     >>>>>>> this call.
>
>     >>>>>>>
>
>     >>>>>>> Thoughts?
>
>     >>>>>>>
>
>     >>>>>>>
>
>     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
>
>     >>>>>>>>
>
>     >>>>>>>> Thanks a lot for the KIP Michal,
>
>     >>>>>>>>
>
>     >>>>>>>> I was thinking about the four options you proposed in more
>
>     >>>>>>>> details
>
>     >>>>>>>> and
>
>     >>>>>>>> this are my thoughts:
>
>     >>>>>>>>
>
>     >>>>>>>> (A) You argue, that users can still "punctuate" on event-time
>
>     >> via
>
>     >>>>>>>> process(), but I am not sure if this is possible. Note, that
>
>     >>>>>>>> users
>
>     >>>>>>>> only
>
>     >>>>>>>> get record timestamps via context.timestamp(). Thus, users
>
>     >> would
>
>     >>>>>>>> need
>
>     >>>>>>>> to
>
>     >>>>>>>> track the time progress per partition (based on the partitions
>
>     >>>>>>>> they
>
>     >>>>>>>> obverse via context.partition(). (This alone puts a huge burden
>
>     >>>>>>>> on
>
>     >>>>>>>> the
>
>     >>>>>>>> user by itself.) However, users are not notified at startup
>
>     >> what
>
>     >>>>>>>> partitions are assigned, and user are not notified when
>
>     >>>>>>>> partitions
>
>     >>>>>>>> get
>
>     >>>>>>>> revoked. Because this information is not available, it's not
>
>     >>>>>>>> possible
>
>     >>>>>>>> to
>
>     >>>>>>>> "manually advance" stream-time, and thus event-time punctuation
>
>     >>>>>>>> within
>
>     >>>>>>>> process() seems not to be possible -- or do you see a way to
>
>     >> get
>
>     >>>>>>>> it
>
>     >>>>>>>> done? And even if, it might still be too clumsy to use.
>
>     >>>>>>>>
>
>     >>>>>>>> (B) This does not allow to mix both approaches, thus limiting
>
>     >>>>>>>> what
>
>     >>>>>>>> users
>
>     >>>>>>>> can do.
>
>     >>>>>>>>
>
>     >>>>>>>> (C) This should give all flexibility we need. However, just
>
>     >>>>>>>> adding
>
>     >>>>>>>> one
>
>     >>>>>>>> more method seems to be a solution that is too simple (cf my
>
>     >>>>>>>> comments
>
>     >>>>>>>> below).
>
>     >>>>>>>>
>
>     >>>>>>>> (D) This might be hard to use. Also, I am not sure how a user
>
>     >>>>>>>> could
>
>     >>>>>>>> enable system-time and event-time punctuation in parallel.
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> Overall options (C) seems to be the most promising approach to
>
>     >>>>>>>> me.
>
>     >>>>>>>> Because I also favor a clean API, we might keep current
>
>     >>>>>>>> punctuate()
>
>     >>>>>>>> as-is, but deprecate it -- so we can remove it at some later
>
>     >>>>>>>> point
>
>     >>>>>>>> when
>
>     >>>>>>>> people use the "new punctuate API".
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> Couple of follow up questions:
>
>     >>>>>>>>
>
>     >>>>>>>> - I am wondering, if we should have two callback methods or
>
>     >> just
>
>     >>>>>>>> one
>
>     >>>>>>>> (ie, a unified for system and event time punctuation or one for
>
>     >>>>>>>> each?).
>
>     >>>>>>>>
>
>     >>>>>>>> - If we have one, how can the user figure out, which condition
>
>     >>>>>>>> did
>
>     >>>>>>>> trigger?
>
>     >>>>>>>>
>
>     >>>>>>>> - How would the API look like, for registering different
>
>     >>>>>>>> punctuate
>
>     >>>>>>>> schedules? The "type" must be somehow defined?
>
>     >>>>>>>>
>
>     >>>>>>>> - We might want to add "complex" schedules later on (like,
>
>     >>>>>>>> punctuate
>
>     >>>>>>>> on
>
>     >>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever
>
>     >>>>>>>> comes
>
>     >>>>>>>> first). I don't say we should add this right away, but we might
>
>     >>>>>>>> want
>
>     >>>>>>>> to
>
>     >>>>>>>> define the API in a way, that it allows extensions like this
>
>     >>>>>>>> later
>
>     >>>>>>>> on,
>
>     >>>>>>>> without redesigning the API (ie, the API should be designed
>
>     >>>>>>>> extensible)
>
>     >>>>>>>>
>
>     >>>>>>>> - Did you ever consider count-based punctuation?
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> I understand, that you would like to solve a simple problem,
>
>     >> but
>
>     >>>>>>>> we
>
>     >>>>>>>> learned from the past, that just "adding some API" quickly
>
>     >> leads
>
>     >>>>>>>> to a
>
>     >>>>>>>> not very well defined API that needs time consuming clean up
>
>     >>>>>>>> later on
>
>     >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
>
>     >>>>>>>> punctuation
>
>     >>>>>>>> KIP
>
>     >>>>>>>> with this from the beginning on to avoid later painful
>
>     >> redesign.
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> -Matthias
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>
>     >>>>>>>>>
>
>     >>>>>>>>>
>
>     >>>>>>>>> Thanks Thomas,
>
>     >>>>>>>>>
>
>     >>>>>>>>> I'm also wary of changing the existing semantics of
>
>     >> punctuate,
>
>     >>>>>>>>> for
>
>     >>>>>>>>> backward compatibility reasons, although I like the
>
>     >> conceptual
>
>     >>>>>>>>> simplicity of that option.
>
>     >>>>>>>>>
>
>     >>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.
>
>     >> I
>
>     >>>>>>>>> added
>
>     >>>>>>>>> this to the KIP now as option (C).
>
>     >>>>>>>>>
>
>     >>>>>>>>> The TimestampExtractor mechanism is actually more flexible,
>
>     >> as
>
>     >>>>>>>>> it
>
>     >>>>>>>>> allows
>
>     >>>>>>>>> you to return any value, you're not limited to event time or
>
>     >>>>>>>>> system
>
>     >>>>>>>>> time
>
>     >>>>>>>>> (although I don't see an actual use case where you might need
>
>     >>>>>>>>> anything
>
>     >>>>>>>>> else then those two). Hence I also proposed the option to
>
>     >> allow
>
>     >>>>>>>>> users
>
>     >>>>>>>>> to, effectively, decide what "stream time" is for them given
>
>     >>>>>>>>> the
>
>     >>>>>>>>> presence or absence of messages, much like they can decide
>
>     >> what
>
>     >>>>>>>>> msg
>
>     >>>>>>>>> time
>
>     >>>>>>>>> means for them using the TimestampExtractor. What do you
>
>     >> think
>
>     >>>>>>>>> about
>
>     >>>>>>>>> that? This is probably most flexible but also most
>
>     >> complicated.
>
>     >>>>>>>>>
>
>     >>>>>>>>> All comments appreciated.
>
>     >>>>>>>>>
>
>     >>>>>>>>> Cheers,
>
>     >>>>>>>>>
>
>     >>>>>>>>> Michal
>
>     >>>>>>>>>
>
>     >>>>>>>>>
>
>     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>
>     >>>>>>>>>>
>
>     >>>>>>>>>>
>
>     >>>>>>>>>> Although I fully agree we need a way to trigger periodic
>
>     >>>>>>>>>> processing
>
>     >>>>>>>>>> that is independent from whether and when messages arrive,
>
>     >>>>>>>>>> I'm
>
>     >>>>>>>>>> not sure
>
>     >>>>>>>>>> I like the idea of changing the existing semantics across
>
>     >> the
>
>     >>>>>>>>>> board.
>
>     >>>>>>>>>> What if we added an additional callback to Processor that
>
>     >> can
>
>     >>>>>>>>>> be
>
>     >>>>>>>>>> scheduled similarly to punctuate() but was always called at
>
>     >>>>>>>>>> fixed, wall
>
>     >>>>>>>>>> clock based intervals? This way you wouldn't have to give
>
>     >> up
>
>     >>>>>>>>>> the
>
>     >>>>>>>>>> notion
>
>     >>>>>>>>>> of stream time to be able to do periodic processing.
>
>     >>>>>>>>>>
>
>     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>> Hi all,
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
>
>     >>>>>>>>>>> semantics
>
>     >>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP- <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>
>     > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>> 138%
>
>     >>>>>>>>>>> 3A+C
>
>     >>>>>>>>>>> hange+
>
>     >>>>>>>>>>> punctuate+semantics>
>
>     >>>>>>>>>>> .
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>> Appreciating there can be different views on system-time
>
>     >> vs
>
>     >>>>>>>>>>> event-
>
>     >>>>>>>>>>> time
>
>     >>>>>>>>>>> semantics for punctuation depending on use-case and the
>
>     >>>>>>>>>>> importance of
>
>     >>>>>>>>>>> backwards compatibility of any such change, I've left it
>
>     >>>>>>>>>>> quite
>
>     >>>>>>>>>>> open
>
>     >>>>>>>>>>> and
>
>     >>>>>>>>>>> hope to fill in more info as the discussion progresses.
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>> Thanks,
>
>     >>>>>>>>>>> Michal
>
>     >>>>>>> --
>
>     >>>>>>>
>
>     >>>>>>>
>
>     >>>>>>>     Tommy Becker
>
>     >>>>>>>
>
>     >>>>>>>     Senior Software Engineer
>
>     >>>>>>>
>
>     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
>
>     >>>>>>>
>
>     >>>>>>>     tivo.com
>
>     >>>>>>>
>
>     >>>>>>>
>
>     >>>>>>> ________________________________
>
>     >>>>>>>
>
>     >>>>>>> This email and any attachments may contain confidential and
>
>     >>>>>>> privileged material for the sole use of the intended recipient.
>
>     >> Any
>
>     >>>>>>> review, copying, or distribution of this email (or any
>
>     >> attachments)
>
>     >>>>>>> by others is prohibited. If you are not the intended recipient,
>
>     >>>>>>> please contact the sender immediately and permanently delete this
>
>     >>>>>>> email and any attachments. No employee or agent of TiVo Inc. is
>
>     >>>>>>> authorized to conclude any binding agreement on behalf of TiVo
>
>     >> Inc.
>
>     >>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a
>
>     >>>>>>> signed written agreement.
>
>     >>>>>>>
>
>     >>>>> --
>
>     >>>>>
>
>     >>>>>
>
>     >>>>>     Tommy Becker
>
>     >>>>>
>
>     >>>>>     Senior Software Engineer
>
>     >>>>>
>
>     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
>
>     >>>>>
>
>     >>>>>     tivo.com
>
>     >>>>>
>
>     >>>>>
>
>     >>>>> ________________________________
>
>     >>>>>
>
>     >>>>> This email and any attachments may contain confidential and
>
>     >> privileged
>
>     >>>>> material for the sole use of the intended recipient. Any review,
>
>     >>> copying,
>
>     >>>>> or distribution of this email (or any attachments) by others is
>
>     >>>> prohibited.
>
>     >>>>> If you are not the intended recipient, please contact the sender
>
>     >>>>> immediately and permanently delete this email and any attachments. No
>
>     >>>>> employee or agent of TiVo Inc. is authorized to conclude any binding
>
>     >>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with
>
>     >> TiVo
>
>     >>>>> Inc. may only be made by a signed written agreement.
>
>     >>>>>
>
>     >>>>
>
>     >>>
>
>     >>
>
>     >
>
>
>
>
>
>
>
>
>
> --
>
> <http://www.openbet.com/>
>
> *Michal Borowiecki*
>
> *Senior Software Engineer L4*
>
> *T: *
>
> +44 208 742 1600 <+44%2020%208742%201600>
>
> +44 203 249 8448 <+44%2020%203249%208448>
>
>
>
> *E: *
>
> michal.borowiecki@openbet.com
>
> *W: *
>
> www.openbet.com
>
> *OpenBet Ltd*
>
> Chiswick Park Building 9
>
> 566 Chiswick High Rd
>
> London
>
> W4 5XT
>
> UK
>
> <https://www.openbet.com/email_promo>
>
>
>
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com and delete it from your system as well as any
> copies. The content of e-mails as well as traffic data may be monitored by
> OpenBet for employment and security purposes. To protect the environment
> please do not print this e-mail unless necessary. OpenBet Ltd. Registered
> Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
> United Kingdom. A company registered in England and Wales. Registered no.
> 3134634. VAT no. GB927523612
>
>
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Tianji Li <sk...@gmail.com>.
Hi Arun,

Could you share your code?

Thanks
Tianji

On Fri, Apr 7, 2017 at 4:21 AM, Arun Mathew <am...@yahoo-corp.jp> wrote:

> Thanks Michal. I don’t have the access yet [arunmathew88]. Should I be
> sending a separate mail for this?
>
> I thought one of the person following this thread would be able to give me
> access.
>
>
>
> *From: *Michal Borowiecki <mi...@openbet.com>
> *Reply-To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> *Date: *Friday, April 7, 2017 at 17:16
> *To: *"dev@kafka.apache.org" <de...@kafka.apache.org>
> *Subject: *Re: [DISCUSS] KIP-138: Change punctuate semantics
>
>
>
> Hi Arun,
>
> I was thinking along the same lines as you, listing the use cases on the
> wiki, but didn't find time to get around doing that yet.
> Don't mind if you do it if you have access now.
> I was thinking it would be nice if, once we have the use cases listed,
> people could use likes to up-vote the use cases similar to what they're
> working on.
>
> I should have a bit more time to action this in the next few days, but
> happy for you to do it if you can beat me to it ;-)
>
> Cheers,
> Michal
>
> On 07/04/17 04:39, Arun Mathew wrote:
>
> Sure, Thanks Matthias. My id is [arunmathew88].
>
>
>
> Of course. I was thinking of a subpage where people can collaborate.
>
>
>
> Will do as per Michael’s suggestion.
>
>
>
> Regards,
>
> Arun Mathew
>
>
>
> On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> <ma...@confluent.io> wrote:
>
>
>
>     Please share your Wiki-ID and a committer can give you write access.
>
>
>
>     Btw: as you did not initiate the KIP, you should not change the KIP
>
>     without the permission of the original author -- in this case Michael.
>
>
>
>     So you might also just share your thought over the mailing list and
>
>     Michael can update the KIP page. Or, as an alternative, just create a
>
>     subpage for the KIP page.
>
>
>
>     @Michael: WDYT?
>
>
>
>
>
>     -Matthias
>
>
>
>
>
>     On 4/6/17 8:05 PM, Arun Mathew wrote:
>
>     > Hi Jay,
>
>     >           Thanks for the advise, I would like to list down the use cases as
>
>     > per your suggestion. But it seems I don't have write permission to the
>
>     > Apache Kafka Confluent Space. Whom shall I request for it?
>
>     >
>
>     > Regarding your last question. We are using a patch in our production system
>
>     > which does exactly this.
>
>     > We window by the event time, but trigger punctuate in <punctuate interval>
>
>     > duration of system time, in the absence of an event crossing the punctuate
>
>     > event time.
>
>     >
>
>     > We are using Kafka Stream for our Audit Trail, where we need to output the
>
>     > event counts on each topic on each cluster aggregated over a 1 minute
>
>     > window. We have to use event time to be able to cross check the counts. But
>
>     > we need to trigger punctuate [aggregate event pushes] by system time in the
>
>     > absence of events. Otherwise the event counts for unexpired windows would
>
>     > be 0 which is bad.
>
>     >
>
>     > "Maybe a hybrid solution works: I window by event time but trigger results
>
>     > by system time for windows that have updated? Not really sure the details
>
>     > of making that work. Does that work? Are there concrete examples where you
>
>     > actually want the current behavior?"
>
>     >
>
>     > --
>
>     > With Regards,
>
>     >
>
>     > Arun Mathew
>
>     > Yahoo! JAPAN Corporation
>
>     >
>
>     > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> <sk...@gmail.com> wrote:
>
>     >
>
>     >> Hi Jay,
>
>     >>
>
>     >> The hybrid solution is exactly what I expect and need for our use cases
>
>     >> when dealing with telecom data.
>
>     >>
>
>     >> Thanks
>
>     >> Tianji
>
>     >>
>
>     >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> <ja...@confluent.io> wrote:
>
>     >>
>
>     >>> Hey guys,
>
>     >>>
>
>     >>> One thing I've always found super important for this kind of design work
>
>     >> is
>
>     >>> to do a really good job of cataloging the landscape of use cases and how
>
>     >>> prevalent each one is. By that I mean not just listing lots of uses, but
>
>     >>> also grouping them into categories that functionally need the same thing.
>
>     >>> In the absence of this it is very hard to reason about design proposals.
>
>     >>> From the proposals so far I think we have a lot of discussion around
>
>     >>> possible apis, but less around what the user needs for different use
>
>     >> cases
>
>     >>> and how they would implement that using the api.
>
>     >>>
>
>     >>> Here is an example:
>
>     >>> You aggregate click and impression data for a reddit like site. Every ten
>
>     >>> minutes you want to output a ranked list of the top 10 articles ranked by
>
>     >>> clicks/impressions for each geographical area. I want to be able run this
>
>     >>> in steady state as well as rerun to regenerate results (or catch up if it
>
>     >>> crashes).
>
>     >>>
>
>     >>> There are a couple of tricky things that seem to make this hard with
>
>     >> either
>
>     >>> of the options proposed:
>
>     >>> 1. If I emit this data using event time I have the problem described
>
>     >> where
>
>     >>> a geographical region with no new clicks or impressions will fail to
>
>     >> output
>
>     >>> results.
>
>     >>> 2. If I emit this data using system time I have the problem that when
>
>     >>> reprocessing data my window may not be ten minutes but 10 hours if my
>
>     >>> processing is very fast so it dramatically changes the output.
>
>     >>>
>
>     >>> Maybe a hybrid solution works: I window by event time but trigger results
>
>     >>> by system time for windows that have updated? Not really sure the details
>
>     >>> of making that work. Does that work? Are there concrete examples where
>
>     >> you
>
>     >>> actually want the current behavior?
>
>     >>>
>
>     >>> -Jay
>
>     >>>
>
>     >>>
>
>     >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com> <ar...@gmail.com>
>
>     >>> wrote:
>
>     >>>
>
>     >>>> Hi All,
>
>     >>>>
>
>     >>>> Thanks for the KIP. We were also in need of a mechanism to trigger
>
>     >>>> punctuate in the absence of events.
>
>     >>>>
>
>     >>>> As I described in [
>
>     >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>
>     >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>
>     >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
>
>     >>>> ],
>
>     >>>>
>
>     >>>>    - Our approached involved using the event time by default.
>
>     >>>>    - The method to check if there is any punctuate ready in the
>
>     >>>>    PunctuationQueue is triggered via the any event received by the
>
>     >> stream
>
>     >>>>    tread, or at the polling intervals in the absence of any events.
>
>     >>>>    - When we create Punctuate objects (which contains the next event
>
>     >> time
>
>     >>>>    for punctuation and interval), we also record the creation time
>
>     >>> (system
>
>     >>>>    time).
>
>     >>>>    - While checking for maturity of Punctuate Schedule by
>
>     >> mayBePunctuate
>
>     >>>>    method, we also check if the system clock has elapsed the punctuate
>
>     >>>>    interval since the schedule creation time.
>
>     >>>>    - In the absence of any event, or in the absence of any event for
>
>     >> one
>
>     >>>>    topic in the partition group assigned to the stream task, the system
>
>     >>>> time
>
>     >>>>    will elapse the interval and we trigger a punctuate using the
>
>     >> expected
>
>     >>>>    punctuation event time.
>
>     >>>>    - we then create the next punctuation schedule as punctuation event
>
>     >>> time
>
>     >>>>    + punctuation interval, [again recording the system time of creation
>
>     >>> of
>
>     >>>> the
>
>     >>>>    schedule].
>
>     >>>>
>
>     >>>> We call this a Hybrid Punctuate. Of course, this approach has pros and
>
>     >>>> cons.
>
>     >>>> Pros
>
>     >>>>
>
>     >>>>    - Punctuates will happen in <punctuate interval> time duration at
>
>     >> max
>
>     >>> in
>
>     >>>>    terms of system time.
>
>     >>>>    - The semantics as a whole continues to revolve around event time.
>
>     >>>>    - We can use the old data [old timestamps] to rerun any experiments
>
>     >> or
>
>     >>>>    tests.
>
>     >>>>
>
>     >>>> Cons
>
>     >>>>
>
>     >>>>    - In case the  <punctuate interval> is not a time duration [say
>
>     >>> logical
>
>     >>>>    time/event count], then the approach might not be meaningful.
>
>     >>>>    - In case there is a case where we have to wait for an actual event
>
>     >>> from
>
>     >>>>    a low event rate partition in the partition group, this approach
>
>     >> will
>
>     >>>> jump
>
>     >>>>    the gun.
>
>     >>>>    - in case the event processing cannot catch up with the event rate
>
>     >> and
>
>     >>>>    the expected timestamp events gets queued for long time, this
>
>     >> approach
>
>     >>>>    might jump the gun.
>
>     >>>>
>
>     >>>> I believe the above approach and discussion goes close to the approach
>
>     >> A.
>
>     >>>>
>
>     >>>> -----------
>
>     >>>>
>
>     >>>> I like the idea of having an even count based punctuate.
>
>     >>>>
>
>     >>>> -----------
>
>     >>>>
>
>     >>>> I agree with the discussion around approach C, that we should provide
>
>     >> the
>
>     >>>> user with the option to choose system time or event time based
>
>     >>> punctuates.
>
>     >>>> But I believe that the user predominantly wants to use event time while
>
>     >>> not
>
>     >>>> missing out on regular punctuates due to event delays or event
>
>     >> absences.
>
>     >>>> Hence a complex punctuate option as Matthias mentioned (quoted below)
>
>     >>> would
>
>     >>>> be most apt.
>
>     >>>>
>
>     >>>> "- We might want to add "complex" schedules later on (like, punctuate
>
>     >> on
>
>     >>>> every 10 seconds event-time or 60 seconds system-time whatever comes
>
>     >>>> first)."
>
>     >>>>
>
>     >>>> -----------
>
>     >>>>
>
>     >>>> I think I read somewhere that Kafka Streams started with System Time as
>
>     >>> the
>
>     >>>> punctuation standard, but was later changed to Event Time. I guess
>
>     >> there
>
>     >>>> would be some good reason behind it. As Kafka Streams want to evolve
>
>     >> more
>
>     >>>> on the Stream Processing front, I believe the emphasis on event time
>
>     >>> would
>
>     >>>> remain quite strong.
>
>     >>>>
>
>     >>>>
>
>     >>>> With Regards,
>
>     >>>>
>
>     >>>> Arun Mathew
>
>     >>>> Yahoo! JAPAN Corporation, Tokyo
>
>     >>>>
>
>     >>>>
>
>     >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com> <to...@tivo.com>
>
>     >> wrote:
>
>     >>>>
>
>     >>>>> Yeah I like PuncutationType much better; I just threw Time out there
>
>     >>>>> more as a strawman than an actual suggestion ;) I still think it's
>
>     >>>>> worth considering what this buys us over an additional callback. I
>
>     >>>>> foresee a number of punctuate implementations following this pattern:
>
>     >>>>>
>
>     >>>>> public void punctuate(PunctuationType type) {
>
>     >>>>>     switch (type) {
>
>     >>>>>         case EVENT_TIME:
>
>     >>>>>             methodA();
>
>     >>>>>             break;
>
>     >>>>>         case SYSTEM_TIME:
>
>     >>>>>             methodB();
>
>     >>>>>             break;
>
>     >>>>>     }
>
>     >>>>> }
>
>     >>>>>
>
>     >>>>> I guess one advantage of this approach is we could add additional
>
>     >>>>> punctuation types later in a backwards compatible way (like event
>
>     >> count
>
>     >>>>> as you mentioned).
>
>     >>>>>
>
>     >>>>> -Tommy
>
>     >>>>>
>
>     >>>>>
>
>     >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
>
>     >>>>>> That sounds promising.
>
>     >>>>>>
>
>     >>>>>> I am just wondering if `Time` is the best name. Maybe we want to
>
>     >> add
>
>     >>>>>> other non-time based punctuations at some point later. I would
>
>     >>>>>> suggest
>
>     >>>>>>
>
>     >>>>>> enum PunctuationType {
>
>     >>>>>>   EVENT_TIME,
>
>     >>>>>>   SYSTEM_TIME,
>
>     >>>>>> }
>
>     >>>>>>
>
>     >>>>>> or similar. Just to keep the door open -- it's easier to add new
>
>     >>>>>> stuff
>
>     >>>>>> if the name is more generic.
>
>     >>>>>>
>
>     >>>>>>
>
>     >>>>>> -Matthias
>
>     >>>>>>
>
>     >>>>>>
>
>     >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>
>     >>>>>>>
>
>     >>>>>>> I agree that the framework providing and managing the notion of
>
>     >>>>>>> stream
>
>     >>>>>>> time is valuable and not something we would want to delegate to
>
>     >> the
>
>     >>>>>>> tasks. I'm not entirely convinced that a separate callback
>
>     >> (option
>
>     >>>>>>> C)
>
>     >>>>>>> is that messy (it could just be a default method with an empty
>
>     >>>>>>> implementation), but if we wanted a single API to handle both
>
>     >>>>>>> cases,
>
>     >>>>>>> how about something like the following?
>
>     >>>>>>>
>
>     >>>>>>> enum Time {
>
>     >>>>>>>    STREAM,
>
>     >>>>>>>    CLOCK
>
>     >>>>>>> }
>
>     >>>>>>>
>
>     >>>>>>> Then on ProcessorContext:
>
>     >>>>>>> context.schedule(Time time, long interval)  // We could allow
>
>     >> this
>
>     >>>>>>> to
>
>     >>>>>>> be called once for each value of time to mix approaches.
>
>     >>>>>>>
>
>     >>>>>>> Then the Processor API becomes:
>
>     >>>>>>> punctuate(Time time) // time here denotes which schedule resulted
>
>     >>>>>>> in
>
>     >>>>>>> this call.
>
>     >>>>>>>
>
>     >>>>>>> Thoughts?
>
>     >>>>>>>
>
>     >>>>>>>
>
>     >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
>
>     >>>>>>>>
>
>     >>>>>>>> Thanks a lot for the KIP Michal,
>
>     >>>>>>>>
>
>     >>>>>>>> I was thinking about the four options you proposed in more
>
>     >>>>>>>> details
>
>     >>>>>>>> and
>
>     >>>>>>>> this are my thoughts:
>
>     >>>>>>>>
>
>     >>>>>>>> (A) You argue, that users can still "punctuate" on event-time
>
>     >> via
>
>     >>>>>>>> process(), but I am not sure if this is possible. Note, that
>
>     >>>>>>>> users
>
>     >>>>>>>> only
>
>     >>>>>>>> get record timestamps via context.timestamp(). Thus, users
>
>     >> would
>
>     >>>>>>>> need
>
>     >>>>>>>> to
>
>     >>>>>>>> track the time progress per partition (based on the partitions
>
>     >>>>>>>> they
>
>     >>>>>>>> obverse via context.partition(). (This alone puts a huge burden
>
>     >>>>>>>> on
>
>     >>>>>>>> the
>
>     >>>>>>>> user by itself.) However, users are not notified at startup
>
>     >> what
>
>     >>>>>>>> partitions are assigned, and user are not notified when
>
>     >>>>>>>> partitions
>
>     >>>>>>>> get
>
>     >>>>>>>> revoked. Because this information is not available, it's not
>
>     >>>>>>>> possible
>
>     >>>>>>>> to
>
>     >>>>>>>> "manually advance" stream-time, and thus event-time punctuation
>
>     >>>>>>>> within
>
>     >>>>>>>> process() seems not to be possible -- or do you see a way to
>
>     >> get
>
>     >>>>>>>> it
>
>     >>>>>>>> done? And even if, it might still be too clumsy to use.
>
>     >>>>>>>>
>
>     >>>>>>>> (B) This does not allow to mix both approaches, thus limiting
>
>     >>>>>>>> what
>
>     >>>>>>>> users
>
>     >>>>>>>> can do.
>
>     >>>>>>>>
>
>     >>>>>>>> (C) This should give all flexibility we need. However, just
>
>     >>>>>>>> adding
>
>     >>>>>>>> one
>
>     >>>>>>>> more method seems to be a solution that is too simple (cf my
>
>     >>>>>>>> comments
>
>     >>>>>>>> below).
>
>     >>>>>>>>
>
>     >>>>>>>> (D) This might be hard to use. Also, I am not sure how a user
>
>     >>>>>>>> could
>
>     >>>>>>>> enable system-time and event-time punctuation in parallel.
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> Overall options (C) seems to be the most promising approach to
>
>     >>>>>>>> me.
>
>     >>>>>>>> Because I also favor a clean API, we might keep current
>
>     >>>>>>>> punctuate()
>
>     >>>>>>>> as-is, but deprecate it -- so we can remove it at some later
>
>     >>>>>>>> point
>
>     >>>>>>>> when
>
>     >>>>>>>> people use the "new punctuate API".
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> Couple of follow up questions:
>
>     >>>>>>>>
>
>     >>>>>>>> - I am wondering, if we should have two callback methods or
>
>     >> just
>
>     >>>>>>>> one
>
>     >>>>>>>> (ie, a unified for system and event time punctuation or one for
>
>     >>>>>>>> each?).
>
>     >>>>>>>>
>
>     >>>>>>>> - If we have one, how can the user figure out, which condition
>
>     >>>>>>>> did
>
>     >>>>>>>> trigger?
>
>     >>>>>>>>
>
>     >>>>>>>> - How would the API look like, for registering different
>
>     >>>>>>>> punctuate
>
>     >>>>>>>> schedules? The "type" must be somehow defined?
>
>     >>>>>>>>
>
>     >>>>>>>> - We might want to add "complex" schedules later on (like,
>
>     >>>>>>>> punctuate
>
>     >>>>>>>> on
>
>     >>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever
>
>     >>>>>>>> comes
>
>     >>>>>>>> first). I don't say we should add this right away, but we might
>
>     >>>>>>>> want
>
>     >>>>>>>> to
>
>     >>>>>>>> define the API in a way, that it allows extensions like this
>
>     >>>>>>>> later
>
>     >>>>>>>> on,
>
>     >>>>>>>> without redesigning the API (ie, the API should be designed
>
>     >>>>>>>> extensible)
>
>     >>>>>>>>
>
>     >>>>>>>> - Did you ever consider count-based punctuation?
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> I understand, that you would like to solve a simple problem,
>
>     >> but
>
>     >>>>>>>> we
>
>     >>>>>>>> learned from the past, that just "adding some API" quickly
>
>     >> leads
>
>     >>>>>>>> to a
>
>     >>>>>>>> not very well defined API that needs time consuming clean up
>
>     >>>>>>>> later on
>
>     >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
>
>     >>>>>>>> punctuation
>
>     >>>>>>>> KIP
>
>     >>>>>>>> with this from the beginning on to avoid later painful
>
>     >> redesign.
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> -Matthias
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>>
>
>     >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>
>     >>>>>>>>>
>
>     >>>>>>>>>
>
>     >>>>>>>>> Thanks Thomas,
>
>     >>>>>>>>>
>
>     >>>>>>>>> I'm also wary of changing the existing semantics of
>
>     >> punctuate,
>
>     >>>>>>>>> for
>
>     >>>>>>>>> backward compatibility reasons, although I like the
>
>     >> conceptual
>
>     >>>>>>>>> simplicity of that option.
>
>     >>>>>>>>>
>
>     >>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.
>
>     >> I
>
>     >>>>>>>>> added
>
>     >>>>>>>>> this to the KIP now as option (C).
>
>     >>>>>>>>>
>
>     >>>>>>>>> The TimestampExtractor mechanism is actually more flexible,
>
>     >> as
>
>     >>>>>>>>> it
>
>     >>>>>>>>> allows
>
>     >>>>>>>>> you to return any value, you're not limited to event time or
>
>     >>>>>>>>> system
>
>     >>>>>>>>> time
>
>     >>>>>>>>> (although I don't see an actual use case where you might need
>
>     >>>>>>>>> anything
>
>     >>>>>>>>> else then those two). Hence I also proposed the option to
>
>     >> allow
>
>     >>>>>>>>> users
>
>     >>>>>>>>> to, effectively, decide what "stream time" is for them given
>
>     >>>>>>>>> the
>
>     >>>>>>>>> presence or absence of messages, much like they can decide
>
>     >> what
>
>     >>>>>>>>> msg
>
>     >>>>>>>>> time
>
>     >>>>>>>>> means for them using the TimestampExtractor. What do you
>
>     >> think
>
>     >>>>>>>>> about
>
>     >>>>>>>>> that? This is probably most flexible but also most
>
>     >> complicated.
>
>     >>>>>>>>>
>
>     >>>>>>>>> All comments appreciated.
>
>     >>>>>>>>>
>
>     >>>>>>>>> Cheers,
>
>     >>>>>>>>>
>
>     >>>>>>>>> Michal
>
>     >>>>>>>>>
>
>     >>>>>>>>>
>
>     >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>
>     >>>>>>>>>>
>
>     >>>>>>>>>>
>
>     >>>>>>>>>> Although I fully agree we need a way to trigger periodic
>
>     >>>>>>>>>> processing
>
>     >>>>>>>>>> that is independent from whether and when messages arrive,
>
>     >>>>>>>>>> I'm
>
>     >>>>>>>>>> not sure
>
>     >>>>>>>>>> I like the idea of changing the existing semantics across
>
>     >> the
>
>     >>>>>>>>>> board.
>
>     >>>>>>>>>> What if we added an additional callback to Processor that
>
>     >> can
>
>     >>>>>>>>>> be
>
>     >>>>>>>>>> scheduled similarly to punctuate() but was always called at
>
>     >>>>>>>>>> fixed, wall
>
>     >>>>>>>>>> clock based intervals? This way you wouldn't have to give
>
>     >> up
>
>     >>>>>>>>>> the
>
>     >>>>>>>>>> notion
>
>     >>>>>>>>>> of stream time to be able to do periodic processing.
>
>     >>>>>>>>>>
>
>     >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>> Hi all,
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
>
>     >>>>>>>>>>> semantics
>
>     >>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP- <https://cwiki.apache.org/confluence/display/KAFKA/KIP->
>
>     > <https://cwiki.apache.org/confluence/display/KAFKA/KIP->>> 138%
>
>     >>>>>>>>>>> 3A+C
>
>     >>>>>>>>>>> hange+
>
>     >>>>>>>>>>> punctuate+semantics>
>
>     >>>>>>>>>>> .
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>> Appreciating there can be different views on system-time
>
>     >> vs
>
>     >>>>>>>>>>> event-
>
>     >>>>>>>>>>> time
>
>     >>>>>>>>>>> semantics for punctuation depending on use-case and the
>
>     >>>>>>>>>>> importance of
>
>     >>>>>>>>>>> backwards compatibility of any such change, I've left it
>
>     >>>>>>>>>>> quite
>
>     >>>>>>>>>>> open
>
>     >>>>>>>>>>> and
>
>     >>>>>>>>>>> hope to fill in more info as the discussion progresses.
>
>     >>>>>>>>>>>
>
>     >>>>>>>>>>> Thanks,
>
>     >>>>>>>>>>> Michal
>
>     >>>>>>> --
>
>     >>>>>>>
>
>     >>>>>>>
>
>     >>>>>>>     Tommy Becker
>
>     >>>>>>>
>
>     >>>>>>>     Senior Software Engineer
>
>     >>>>>>>
>
>     >>>>>>>     O +1 919.460.4747 <(919)%20460-4747>
>
>     >>>>>>>
>
>     >>>>>>>     tivo.com
>
>     >>>>>>>
>
>     >>>>>>>
>
>     >>>>>>> ________________________________
>
>     >>>>>>>
>
>     >>>>>>> This email and any attachments may contain confidential and
>
>     >>>>>>> privileged material for the sole use of the intended recipient.
>
>     >> Any
>
>     >>>>>>> review, copying, or distribution of this email (or any
>
>     >> attachments)
>
>     >>>>>>> by others is prohibited. If you are not the intended recipient,
>
>     >>>>>>> please contact the sender immediately and permanently delete this
>
>     >>>>>>> email and any attachments. No employee or agent of TiVo Inc. is
>
>     >>>>>>> authorized to conclude any binding agreement on behalf of TiVo
>
>     >> Inc.
>
>     >>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a
>
>     >>>>>>> signed written agreement.
>
>     >>>>>>>
>
>     >>>>> --
>
>     >>>>>
>
>     >>>>>
>
>     >>>>>     Tommy Becker
>
>     >>>>>
>
>     >>>>>     Senior Software Engineer
>
>     >>>>>
>
>     >>>>>     O +1 919.460.4747 <(919)%20460-4747>
>
>     >>>>>
>
>     >>>>>     tivo.com
>
>     >>>>>
>
>     >>>>>
>
>     >>>>> ________________________________
>
>     >>>>>
>
>     >>>>> This email and any attachments may contain confidential and
>
>     >> privileged
>
>     >>>>> material for the sole use of the intended recipient. Any review,
>
>     >>> copying,
>
>     >>>>> or distribution of this email (or any attachments) by others is
>
>     >>>> prohibited.
>
>     >>>>> If you are not the intended recipient, please contact the sender
>
>     >>>>> immediately and permanently delete this email and any attachments. No
>
>     >>>>> employee or agent of TiVo Inc. is authorized to conclude any binding
>
>     >>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with
>
>     >> TiVo
>
>     >>>>> Inc. may only be made by a signed written agreement.
>
>     >>>>>
>
>     >>>>
>
>     >>>
>
>     >>
>
>     >
>
>
>
>
>
>
>
>
>
> --
>
> <http://www.openbet.com/>
>
> *Michal Borowiecki*
>
> *Senior Software Engineer L4*
>
> *T: *
>
> +44 208 742 1600 <+44%2020%208742%201600>
>
> +44 203 249 8448 <+44%2020%203249%208448>
>
>
>
> *E: *
>
> michal.borowiecki@openbet.com
>
> *W: *
>
> www.openbet.com
>
> *OpenBet Ltd*
>
> Chiswick Park Building 9
>
> 566 Chiswick High Rd
>
> London
>
> W4 5XT
>
> UK
>
> <https://www.openbet.com/email_promo>
>
>
>
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com and delete it from your system as well as any
> copies. The content of e-mails as well as traffic data may be monitored by
> OpenBet for employment and security purposes. To protect the environment
> please do not print this e-mail unless necessary. OpenBet Ltd. Registered
> Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
> United Kingdom. A company registered in England and Wales. Registered no.
> 3134634. VAT no. GB927523612
>
>
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Arun Mathew <am...@yahoo-corp.jp>.
Thanks Michal. I don’t have the access yet [arunmathew88]. Should I be sending a separate mail for this?
I thought one of the person following this thread would be able to give me access.

From: Michal Borowiecki <mi...@openbet.com>
Reply-To: "dev@kafka.apache.org" <de...@kafka.apache.org>
Date: Friday, April 7, 2017 at 17:16
To: "dev@kafka.apache.org" <de...@kafka.apache.org>
Subject: Re: [DISCUSS] KIP-138: Change punctuate semantics

Hi Arun,

I was thinking along the same lines as you, listing the use cases on the wiki, but didn't find time to get around doing that yet.
Don't mind if you do it if you have access now.
I was thinking it would be nice if, once we have the use cases listed, people could use likes to up-vote the use cases similar to what they're working on.

I should have a bit more time to action this in the next few days, but happy for you to do it if you can beat me to it ;-)

Cheers,
Michal
On 07/04/17 04:39, Arun Mathew wrote:

Sure, Thanks Matthias. My id is [arunmathew88].



Of course. I was thinking of a subpage where people can collaborate.



Will do as per Michael’s suggestion.



Regards,

Arun Mathew



On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> wrote:



    Please share your Wiki-ID and a committer can give you write access.



    Btw: as you did not initiate the KIP, you should not change the KIP

    without the permission of the original author -- in this case Michael.



    So you might also just share your thought over the mailing list and

    Michael can update the KIP page. Or, as an alternative, just create a

    subpage for the KIP page.



    @Michael: WDYT?





    -Matthias





    On 4/6/17 8:05 PM, Arun Mathew wrote:

    > Hi Jay,

    >           Thanks for the advise, I would like to list down the use cases as

    > per your suggestion. But it seems I don't have write permission to the

    > Apache Kafka Confluent Space. Whom shall I request for it?

    >

    > Regarding your last question. We are using a patch in our production system

    > which does exactly this.

    > We window by the event time, but trigger punctuate in <punctuate interval>

    > duration of system time, in the absence of an event crossing the punctuate

    > event time.

    >

    > We are using Kafka Stream for our Audit Trail, where we need to output the

    > event counts on each topic on each cluster aggregated over a 1 minute

    > window. We have to use event time to be able to cross check the counts. But

    > we need to trigger punctuate [aggregate event pushes] by system time in the

    > absence of events. Otherwise the event counts for unexpired windows would

    > be 0 which is bad.

    >

    > "Maybe a hybrid solution works: I window by event time but trigger results

    > by system time for windows that have updated? Not really sure the details

    > of making that work. Does that work? Are there concrete examples where you

    > actually want the current behavior?"

    >

    > --

    > With Regards,

    >

    > Arun Mathew

    > Yahoo! JAPAN Corporation

    >

    > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> wrote:

    >

    >> Hi Jay,

    >>

    >> The hybrid solution is exactly what I expect and need for our use cases

    >> when dealing with telecom data.

    >>

    >> Thanks

    >> Tianji

    >>

    >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> wrote:

    >>

    >>> Hey guys,

    >>>

    >>> One thing I've always found super important for this kind of design work

    >> is

    >>> to do a really good job of cataloging the landscape of use cases and how

    >>> prevalent each one is. By that I mean not just listing lots of uses, but

    >>> also grouping them into categories that functionally need the same thing.

    >>> In the absence of this it is very hard to reason about design proposals.

    >>> From the proposals so far I think we have a lot of discussion around

    >>> possible apis, but less around what the user needs for different use

    >> cases

    >>> and how they would implement that using the api.

    >>>

    >>> Here is an example:

    >>> You aggregate click and impression data for a reddit like site. Every ten

    >>> minutes you want to output a ranked list of the top 10 articles ranked by

    >>> clicks/impressions for each geographical area. I want to be able run this

    >>> in steady state as well as rerun to regenerate results (or catch up if it

    >>> crashes).

    >>>

    >>> There are a couple of tricky things that seem to make this hard with

    >> either

    >>> of the options proposed:

    >>> 1. If I emit this data using event time I have the problem described

    >> where

    >>> a geographical region with no new clicks or impressions will fail to

    >> output

    >>> results.

    >>> 2. If I emit this data using system time I have the problem that when

    >>> reprocessing data my window may not be ten minutes but 10 hours if my

    >>> processing is very fast so it dramatically changes the output.

    >>>

    >>> Maybe a hybrid solution works: I window by event time but trigger results

    >>> by system time for windows that have updated? Not really sure the details

    >>> of making that work. Does that work? Are there concrete examples where

    >> you

    >>> actually want the current behavior?

    >>>

    >>> -Jay

    >>>

    >>>

    >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com>

    >>> wrote:

    >>>

    >>>> Hi All,

    >>>>

    >>>> Thanks for the KIP. We were also in need of a mechanism to trigger

    >>>> punctuate in the absence of events.

    >>>>

    >>>> As I described in [

    >>>> https://issues.apache.org/jira/browse/KAFKA-3514?

    >>>> focusedCommentId=15926036&page=com.atlassian.jira.

    >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036

    >>>> ],

    >>>>

    >>>>    - Our approached involved using the event time by default.

    >>>>    - The method to check if there is any punctuate ready in the

    >>>>    PunctuationQueue is triggered via the any event received by the

    >> stream

    >>>>    tread, or at the polling intervals in the absence of any events.

    >>>>    - When we create Punctuate objects (which contains the next event

    >> time

    >>>>    for punctuation and interval), we also record the creation time

    >>> (system

    >>>>    time).

    >>>>    - While checking for maturity of Punctuate Schedule by

    >> mayBePunctuate

    >>>>    method, we also check if the system clock has elapsed the punctuate

    >>>>    interval since the schedule creation time.

    >>>>    - In the absence of any event, or in the absence of any event for

    >> one

    >>>>    topic in the partition group assigned to the stream task, the system

    >>>> time

    >>>>    will elapse the interval and we trigger a punctuate using the

    >> expected

    >>>>    punctuation event time.

    >>>>    - we then create the next punctuation schedule as punctuation event

    >>> time

    >>>>    + punctuation interval, [again recording the system time of creation

    >>> of

    >>>> the

    >>>>    schedule].

    >>>>

    >>>> We call this a Hybrid Punctuate. Of course, this approach has pros and

    >>>> cons.

    >>>> Pros

    >>>>

    >>>>    - Punctuates will happen in <punctuate interval> time duration at

    >> max

    >>> in

    >>>>    terms of system time.

    >>>>    - The semantics as a whole continues to revolve around event time.

    >>>>    - We can use the old data [old timestamps] to rerun any experiments

    >> or

    >>>>    tests.

    >>>>

    >>>> Cons

    >>>>

    >>>>    - In case the  <punctuate interval> is not a time duration [say

    >>> logical

    >>>>    time/event count], then the approach might not be meaningful.

    >>>>    - In case there is a case where we have to wait for an actual event

    >>> from

    >>>>    a low event rate partition in the partition group, this approach

    >> will

    >>>> jump

    >>>>    the gun.

    >>>>    - in case the event processing cannot catch up with the event rate

    >> and

    >>>>    the expected timestamp events gets queued for long time, this

    >> approach

    >>>>    might jump the gun.

    >>>>

    >>>> I believe the above approach and discussion goes close to the approach

    >> A.

    >>>>

    >>>> -----------

    >>>>

    >>>> I like the idea of having an even count based punctuate.

    >>>>

    >>>> -----------

    >>>>

    >>>> I agree with the discussion around approach C, that we should provide

    >> the

    >>>> user with the option to choose system time or event time based

    >>> punctuates.

    >>>> But I believe that the user predominantly wants to use event time while

    >>> not

    >>>> missing out on regular punctuates due to event delays or event

    >> absences.

    >>>> Hence a complex punctuate option as Matthias mentioned (quoted below)

    >>> would

    >>>> be most apt.

    >>>>

    >>>> "- We might want to add "complex" schedules later on (like, punctuate

    >> on

    >>>> every 10 seconds event-time or 60 seconds system-time whatever comes

    >>>> first)."

    >>>>

    >>>> -----------

    >>>>

    >>>> I think I read somewhere that Kafka Streams started with System Time as

    >>> the

    >>>> punctuation standard, but was later changed to Event Time. I guess

    >> there

    >>>> would be some good reason behind it. As Kafka Streams want to evolve

    >> more

    >>>> on the Stream Processing front, I believe the emphasis on event time

    >>> would

    >>>> remain quite strong.

    >>>>

    >>>>

    >>>> With Regards,

    >>>>

    >>>> Arun Mathew

    >>>> Yahoo! JAPAN Corporation, Tokyo

    >>>>

    >>>>

    >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com>

    >> wrote:

    >>>>

    >>>>> Yeah I like PuncutationType much better; I just threw Time out there

    >>>>> more as a strawman than an actual suggestion ;) I still think it's

    >>>>> worth considering what this buys us over an additional callback. I

    >>>>> foresee a number of punctuate implementations following this pattern:

    >>>>>

    >>>>> public void punctuate(PunctuationType type) {

    >>>>>     switch (type) {

    >>>>>         case EVENT_TIME:

    >>>>>             methodA();

    >>>>>             break;

    >>>>>         case SYSTEM_TIME:

    >>>>>             methodB();

    >>>>>             break;

    >>>>>     }

    >>>>> }

    >>>>>

    >>>>> I guess one advantage of this approach is we could add additional

    >>>>> punctuation types later in a backwards compatible way (like event

    >> count

    >>>>> as you mentioned).

    >>>>>

    >>>>> -Tommy

    >>>>>

    >>>>>

    >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:

    >>>>>> That sounds promising.

    >>>>>>

    >>>>>> I am just wondering if `Time` is the best name. Maybe we want to

    >> add

    >>>>>> other non-time based punctuations at some point later. I would

    >>>>>> suggest

    >>>>>>

    >>>>>> enum PunctuationType {

    >>>>>>   EVENT_TIME,

    >>>>>>   SYSTEM_TIME,

    >>>>>> }

    >>>>>>

    >>>>>> or similar. Just to keep the door open -- it's easier to add new

    >>>>>> stuff

    >>>>>> if the name is more generic.

    >>>>>>

    >>>>>>

    >>>>>> -Matthias

    >>>>>>

    >>>>>>

    >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:

    >>>>>>>

    >>>>>>> I agree that the framework providing and managing the notion of

    >>>>>>> stream

    >>>>>>> time is valuable and not something we would want to delegate to

    >> the

    >>>>>>> tasks. I'm not entirely convinced that a separate callback

    >> (option

    >>>>>>> C)

    >>>>>>> is that messy (it could just be a default method with an empty

    >>>>>>> implementation), but if we wanted a single API to handle both

    >>>>>>> cases,

    >>>>>>> how about something like the following?

    >>>>>>>

    >>>>>>> enum Time {

    >>>>>>>    STREAM,

    >>>>>>>    CLOCK

    >>>>>>> }

    >>>>>>>

    >>>>>>> Then on ProcessorContext:

    >>>>>>> context.schedule(Time time, long interval)  // We could allow

    >> this

    >>>>>>> to

    >>>>>>> be called once for each value of time to mix approaches.

    >>>>>>>

    >>>>>>> Then the Processor API becomes:

    >>>>>>> punctuate(Time time) // time here denotes which schedule resulted

    >>>>>>> in

    >>>>>>> this call.

    >>>>>>>

    >>>>>>> Thoughts?

    >>>>>>>

    >>>>>>>

    >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:

    >>>>>>>>

    >>>>>>>> Thanks a lot for the KIP Michal,

    >>>>>>>>

    >>>>>>>> I was thinking about the four options you proposed in more

    >>>>>>>> details

    >>>>>>>> and

    >>>>>>>> this are my thoughts:

    >>>>>>>>

    >>>>>>>> (A) You argue, that users can still "punctuate" on event-time

    >> via

    >>>>>>>> process(), but I am not sure if this is possible. Note, that

    >>>>>>>> users

    >>>>>>>> only

    >>>>>>>> get record timestamps via context.timestamp(). Thus, users

    >> would

    >>>>>>>> need

    >>>>>>>> to

    >>>>>>>> track the time progress per partition (based on the partitions

    >>>>>>>> they

    >>>>>>>> obverse via context.partition(). (This alone puts a huge burden

    >>>>>>>> on

    >>>>>>>> the

    >>>>>>>> user by itself.) However, users are not notified at startup

    >> what

    >>>>>>>> partitions are assigned, and user are not notified when

    >>>>>>>> partitions

    >>>>>>>> get

    >>>>>>>> revoked. Because this information is not available, it's not

    >>>>>>>> possible

    >>>>>>>> to

    >>>>>>>> "manually advance" stream-time, and thus event-time punctuation

    >>>>>>>> within

    >>>>>>>> process() seems not to be possible -- or do you see a way to

    >> get

    >>>>>>>> it

    >>>>>>>> done? And even if, it might still be too clumsy to use.

    >>>>>>>>

    >>>>>>>> (B) This does not allow to mix both approaches, thus limiting

    >>>>>>>> what

    >>>>>>>> users

    >>>>>>>> can do.

    >>>>>>>>

    >>>>>>>> (C) This should give all flexibility we need. However, just

    >>>>>>>> adding

    >>>>>>>> one

    >>>>>>>> more method seems to be a solution that is too simple (cf my

    >>>>>>>> comments

    >>>>>>>> below).

    >>>>>>>>

    >>>>>>>> (D) This might be hard to use. Also, I am not sure how a user

    >>>>>>>> could

    >>>>>>>> enable system-time and event-time punctuation in parallel.

    >>>>>>>>

    >>>>>>>>

    >>>>>>>>

    >>>>>>>> Overall options (C) seems to be the most promising approach to

    >>>>>>>> me.

    >>>>>>>> Because I also favor a clean API, we might keep current

    >>>>>>>> punctuate()

    >>>>>>>> as-is, but deprecate it -- so we can remove it at some later

    >>>>>>>> point

    >>>>>>>> when

    >>>>>>>> people use the "new punctuate API".

    >>>>>>>>

    >>>>>>>>

    >>>>>>>> Couple of follow up questions:

    >>>>>>>>

    >>>>>>>> - I am wondering, if we should have two callback methods or

    >> just

    >>>>>>>> one

    >>>>>>>> (ie, a unified for system and event time punctuation or one for

    >>>>>>>> each?).

    >>>>>>>>

    >>>>>>>> - If we have one, how can the user figure out, which condition

    >>>>>>>> did

    >>>>>>>> trigger?

    >>>>>>>>

    >>>>>>>> - How would the API look like, for registering different

    >>>>>>>> punctuate

    >>>>>>>> schedules? The "type" must be somehow defined?

    >>>>>>>>

    >>>>>>>> - We might want to add "complex" schedules later on (like,

    >>>>>>>> punctuate

    >>>>>>>> on

    >>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever

    >>>>>>>> comes

    >>>>>>>> first). I don't say we should add this right away, but we might

    >>>>>>>> want

    >>>>>>>> to

    >>>>>>>> define the API in a way, that it allows extensions like this

    >>>>>>>> later

    >>>>>>>> on,

    >>>>>>>> without redesigning the API (ie, the API should be designed

    >>>>>>>> extensible)

    >>>>>>>>

    >>>>>>>> - Did you ever consider count-based punctuation?

    >>>>>>>>

    >>>>>>>>

    >>>>>>>> I understand, that you would like to solve a simple problem,

    >> but

    >>>>>>>> we

    >>>>>>>> learned from the past, that just "adding some API" quickly

    >> leads

    >>>>>>>> to a

    >>>>>>>> not very well defined API that needs time consuming clean up

    >>>>>>>> later on

    >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic

    >>>>>>>> punctuation

    >>>>>>>> KIP

    >>>>>>>> with this from the beginning on to avoid later painful

    >> redesign.

    >>>>>>>>

    >>>>>>>>

    >>>>>>>>

    >>>>>>>> -Matthias

    >>>>>>>>

    >>>>>>>>

    >>>>>>>>

    >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:

    >>>>>>>>>

    >>>>>>>>>

    >>>>>>>>> Thanks Thomas,

    >>>>>>>>>

    >>>>>>>>> I'm also wary of changing the existing semantics of

    >> punctuate,

    >>>>>>>>> for

    >>>>>>>>> backward compatibility reasons, although I like the

    >> conceptual

    >>>>>>>>> simplicity of that option.

    >>>>>>>>>

    >>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.

    >> I

    >>>>>>>>> added

    >>>>>>>>> this to the KIP now as option (C).

    >>>>>>>>>

    >>>>>>>>> The TimestampExtractor mechanism is actually more flexible,

    >> as

    >>>>>>>>> it

    >>>>>>>>> allows

    >>>>>>>>> you to return any value, you're not limited to event time or

    >>>>>>>>> system

    >>>>>>>>> time

    >>>>>>>>> (although I don't see an actual use case where you might need

    >>>>>>>>> anything

    >>>>>>>>> else then those two). Hence I also proposed the option to

    >> allow

    >>>>>>>>> users

    >>>>>>>>> to, effectively, decide what "stream time" is for them given

    >>>>>>>>> the

    >>>>>>>>> presence or absence of messages, much like they can decide

    >> what

    >>>>>>>>> msg

    >>>>>>>>> time

    >>>>>>>>> means for them using the TimestampExtractor. What do you

    >> think

    >>>>>>>>> about

    >>>>>>>>> that? This is probably most flexible but also most

    >> complicated.

    >>>>>>>>>

    >>>>>>>>> All comments appreciated.

    >>>>>>>>>

    >>>>>>>>> Cheers,

    >>>>>>>>>

    >>>>>>>>> Michal

    >>>>>>>>>

    >>>>>>>>>

    >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:

    >>>>>>>>>>

    >>>>>>>>>>

    >>>>>>>>>> Although I fully agree we need a way to trigger periodic

    >>>>>>>>>> processing

    >>>>>>>>>> that is independent from whether and when messages arrive,

    >>>>>>>>>> I'm

    >>>>>>>>>> not sure

    >>>>>>>>>> I like the idea of changing the existing semantics across

    >> the

    >>>>>>>>>> board.

    >>>>>>>>>> What if we added an additional callback to Processor that

    >> can

    >>>>>>>>>> be

    >>>>>>>>>> scheduled similarly to punctuate() but was always called at

    >>>>>>>>>> fixed, wall

    >>>>>>>>>> clock based intervals? This way you wouldn't have to give

    >> up

    >>>>>>>>>> the

    >>>>>>>>>> notion

    >>>>>>>>>> of stream time to be able to do periodic processing.

    >>>>>>>>>>

    >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:

    >>>>>>>>>>>

    >>>>>>>>>>>

    >>>>>>>>>>> Hi all,

    >>>>>>>>>>>

    >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate

    >>>>>>>>>>> semantics

    >>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-<https://cwiki.apache.org/confluence/display/KAFKA/KIP->

    ><https://cwiki.apache.org/confluence/display/KAFKA/KIP->>> 138%

    >>>>>>>>>>> 3A+C

    >>>>>>>>>>> hange+

    >>>>>>>>>>> punctuate+semantics>

    >>>>>>>>>>> .

    >>>>>>>>>>>

    >>>>>>>>>>> Appreciating there can be different views on system-time

    >> vs

    >>>>>>>>>>> event-

    >>>>>>>>>>> time

    >>>>>>>>>>> semantics for punctuation depending on use-case and the

    >>>>>>>>>>> importance of

    >>>>>>>>>>> backwards compatibility of any such change, I've left it

    >>>>>>>>>>> quite

    >>>>>>>>>>> open

    >>>>>>>>>>> and

    >>>>>>>>>>> hope to fill in more info as the discussion progresses.

    >>>>>>>>>>>

    >>>>>>>>>>> Thanks,

    >>>>>>>>>>> Michal

    >>>>>>> --

    >>>>>>>

    >>>>>>>

    >>>>>>>     Tommy Becker

    >>>>>>>

    >>>>>>>     Senior Software Engineer

    >>>>>>>

    >>>>>>>     O +1 919.460.4747

    >>>>>>>

    >>>>>>>     tivo.com

    >>>>>>>

    >>>>>>>

    >>>>>>> ________________________________

    >>>>>>>

    >>>>>>> This email and any attachments may contain confidential and

    >>>>>>> privileged material for the sole use of the intended recipient.

    >> Any

    >>>>>>> review, copying, or distribution of this email (or any

    >> attachments)

    >>>>>>> by others is prohibited. If you are not the intended recipient,

    >>>>>>> please contact the sender immediately and permanently delete this

    >>>>>>> email and any attachments. No employee or agent of TiVo Inc. is

    >>>>>>> authorized to conclude any binding agreement on behalf of TiVo

    >> Inc.

    >>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a

    >>>>>>> signed written agreement.

    >>>>>>>

    >>>>> --

    >>>>>

    >>>>>

    >>>>>     Tommy Becker

    >>>>>

    >>>>>     Senior Software Engineer

    >>>>>

    >>>>>     O +1 919.460.4747

    >>>>>

    >>>>>     tivo.com

    >>>>>

    >>>>>

    >>>>> ________________________________

    >>>>>

    >>>>> This email and any attachments may contain confidential and

    >> privileged

    >>>>> material for the sole use of the intended recipient. Any review,

    >>> copying,

    >>>>> or distribution of this email (or any attachments) by others is

    >>>> prohibited.

    >>>>> If you are not the intended recipient, please contact the sender

    >>>>> immediately and permanently delete this email and any attachments. No

    >>>>> employee or agent of TiVo Inc. is authorized to conclude any binding

    >>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with

    >> TiVo

    >>>>> Inc. may only be made by a signed written agreement.

    >>>>>

    >>>>

    >>>

    >>

    >







--

[cid:image001.gif@01D2AFC3.53310250]<http://www.openbet.com/>

Michal Borowiecki

Senior Software Engineer L4



T:

+44 208 742 1600


+44 203 249 8448






E:

michal.borowiecki@openbet.com<ma...@openbet.com>



W:

www.openbet.com<http://www.openbet.com/>




OpenBet Ltd


Chiswick Park Building 9


566 Chiswick High Rd


London


W4 5XT


UK


[cid:image003.gif@01D2AFC3.53310250]<https://www.openbet.com/email_promo>


This message is confidential and intended only for the addressee. If you have received this message in error, please immediately notify the postmaster@openbet.com<ma...@openbet.com> and delete it from your system as well as any copies. The content of e-mails as well as traffic data may be monitored by OpenBet for employment and security purposes. To protect the environment please do not print this e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company registered in England and Wales. Registered no. 3134634. VAT no. GB927523612



Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Hi Arun,

I was thinking along the same lines as you, listing the use cases on the 
wiki, but didn't find time to get around doing that yet.
Don't mind if you do it if you have access now.
I was thinking it would be nice if, once we have the use cases listed, 
people could use likes to up-vote the use cases similar to what they're 
working on.

I should have a bit more time to action this in the next few days, but 
happy for you to do it if you can beat me to it ;-)

Cheers,
Michal

On 07/04/17 04:39, Arun Mathew wrote:
> Sure, Thanks Matthias. My id is [arunmathew88].
>
> Of course. I was thinking of a subpage where people can collaborate.
>
> Will do as per Michael\u2019s suggestion.
>
> Regards,
> Arun Mathew
>
> On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> wrote:
>
>      Please share your Wiki-ID and a committer can give you write access.
>      
>      Btw: as you did not initiate the KIP, you should not change the KIP
>      without the permission of the original author -- in this case Michael.
>      
>      So you might also just share your thought over the mailing list and
>      Michael can update the KIP page. Or, as an alternative, just create a
>      subpage for the KIP page.
>      
>      @Michael: WDYT?
>      
>      
>      -Matthias
>      
>      
>      On 4/6/17 8:05 PM, Arun Mathew wrote:
>      > Hi Jay,
>      >           Thanks for the advise, I would like to list down the use cases as
>      > per your suggestion. But it seems I don't have write permission to the
>      > Apache Kafka Confluent Space. Whom shall I request for it?
>      >
>      > Regarding your last question. We are using a patch in our production system
>      > which does exactly this.
>      > We window by the event time, but trigger punctuate in <punctuate interval>
>      > duration of system time, in the absence of an event crossing the punctuate
>      > event time.
>      >
>      > We are using Kafka Stream for our Audit Trail, where we need to output the
>      > event counts on each topic on each cluster aggregated over a 1 minute
>      > window. We have to use event time to be able to cross check the counts. But
>      > we need to trigger punctuate [aggregate event pushes] by system time in the
>      > absence of events. Otherwise the event counts for unexpired windows would
>      > be 0 which is bad.
>      >
>      > "Maybe a hybrid solution works: I window by event time but trigger results
>      > by system time for windows that have updated? Not really sure the details
>      > of making that work. Does that work? Are there concrete examples where you
>      > actually want the current behavior?"
>      >
>      > --
>      > With Regards,
>      >
>      > Arun Mathew
>      > Yahoo! JAPAN Corporation
>      >
>      > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> wrote:
>      >
>      >> Hi Jay,
>      >>
>      >> The hybrid solution is exactly what I expect and need for our use cases
>      >> when dealing with telecom data.
>      >>
>      >> Thanks
>      >> Tianji
>      >>
>      >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> wrote:
>      >>
>      >>> Hey guys,
>      >>>
>      >>> One thing I've always found super important for this kind of design work
>      >> is
>      >>> to do a really good job of cataloging the landscape of use cases and how
>      >>> prevalent each one is. By that I mean not just listing lots of uses, but
>      >>> also grouping them into categories that functionally need the same thing.
>      >>> In the absence of this it is very hard to reason about design proposals.
>      >>> From the proposals so far I think we have a lot of discussion around
>      >>> possible apis, but less around what the user needs for different use
>      >> cases
>      >>> and how they would implement that using the api.
>      >>>
>      >>> Here is an example:
>      >>> You aggregate click and impression data for a reddit like site. Every ten
>      >>> minutes you want to output a ranked list of the top 10 articles ranked by
>      >>> clicks/impressions for each geographical area. I want to be able run this
>      >>> in steady state as well as rerun to regenerate results (or catch up if it
>      >>> crashes).
>      >>>
>      >>> There are a couple of tricky things that seem to make this hard with
>      >> either
>      >>> of the options proposed:
>      >>> 1. If I emit this data using event time I have the problem described
>      >> where
>      >>> a geographical region with no new clicks or impressions will fail to
>      >> output
>      >>> results.
>      >>> 2. If I emit this data using system time I have the problem that when
>      >>> reprocessing data my window may not be ten minutes but 10 hours if my
>      >>> processing is very fast so it dramatically changes the output.
>      >>>
>      >>> Maybe a hybrid solution works: I window by event time but trigger results
>      >>> by system time for windows that have updated? Not really sure the details
>      >>> of making that work. Does that work? Are there concrete examples where
>      >> you
>      >>> actually want the current behavior?
>      >>>
>      >>> -Jay
>      >>>
>      >>>
>      >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com>
>      >>> wrote:
>      >>>
>      >>>> Hi All,
>      >>>>
>      >>>> Thanks for the KIP. We were also in need of a mechanism to trigger
>      >>>> punctuate in the absence of events.
>      >>>>
>      >>>> As I described in [
>      >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>      >>>> focusedCommentId=15926036&page=com.atlassian.jira.
>      >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
>      >>>> ],
>      >>>>
>      >>>>    - Our approached involved using the event time by default.
>      >>>>    - The method to check if there is any punctuate ready in the
>      >>>>    PunctuationQueue is triggered via the any event received by the
>      >> stream
>      >>>>    tread, or at the polling intervals in the absence of any events.
>      >>>>    - When we create Punctuate objects (which contains the next event
>      >> time
>      >>>>    for punctuation and interval), we also record the creation time
>      >>> (system
>      >>>>    time).
>      >>>>    - While checking for maturity of Punctuate Schedule by
>      >> mayBePunctuate
>      >>>>    method, we also check if the system clock has elapsed the punctuate
>      >>>>    interval since the schedule creation time.
>      >>>>    - In the absence of any event, or in the absence of any event for
>      >> one
>      >>>>    topic in the partition group assigned to the stream task, the system
>      >>>> time
>      >>>>    will elapse the interval and we trigger a punctuate using the
>      >> expected
>      >>>>    punctuation event time.
>      >>>>    - we then create the next punctuation schedule as punctuation event
>      >>> time
>      >>>>    + punctuation interval, [again recording the system time of creation
>      >>> of
>      >>>> the
>      >>>>    schedule].
>      >>>>
>      >>>> We call this a Hybrid Punctuate. Of course, this approach has pros and
>      >>>> cons.
>      >>>> Pros
>      >>>>
>      >>>>    - Punctuates will happen in <punctuate interval> time duration at
>      >> max
>      >>> in
>      >>>>    terms of system time.
>      >>>>    - The semantics as a whole continues to revolve around event time.
>      >>>>    - We can use the old data [old timestamps] to rerun any experiments
>      >> or
>      >>>>    tests.
>      >>>>
>      >>>> Cons
>      >>>>
>      >>>>    - In case the  <punctuate interval> is not a time duration [say
>      >>> logical
>      >>>>    time/event count], then the approach might not be meaningful.
>      >>>>    - In case there is a case where we have to wait for an actual event
>      >>> from
>      >>>>    a low event rate partition in the partition group, this approach
>      >> will
>      >>>> jump
>      >>>>    the gun.
>      >>>>    - in case the event processing cannot catch up with the event rate
>      >> and
>      >>>>    the expected timestamp events gets queued for long time, this
>      >> approach
>      >>>>    might jump the gun.
>      >>>>
>      >>>> I believe the above approach and discussion goes close to the approach
>      >> A.
>      >>>>
>      >>>> -----------
>      >>>>
>      >>>> I like the idea of having an even count based punctuate.
>      >>>>
>      >>>> -----------
>      >>>>
>      >>>> I agree with the discussion around approach C, that we should provide
>      >> the
>      >>>> user with the option to choose system time or event time based
>      >>> punctuates.
>      >>>> But I believe that the user predominantly wants to use event time while
>      >>> not
>      >>>> missing out on regular punctuates due to event delays or event
>      >> absences.
>      >>>> Hence a complex punctuate option as Matthias mentioned (quoted below)
>      >>> would
>      >>>> be most apt.
>      >>>>
>      >>>> "- We might want to add "complex" schedules later on (like, punctuate
>      >> on
>      >>>> every 10 seconds event-time or 60 seconds system-time whatever comes
>      >>>> first)."
>      >>>>
>      >>>> -----------
>      >>>>
>      >>>> I think I read somewhere that Kafka Streams started with System Time as
>      >>> the
>      >>>> punctuation standard, but was later changed to Event Time. I guess
>      >> there
>      >>>> would be some good reason behind it. As Kafka Streams want to evolve
>      >> more
>      >>>> on the Stream Processing front, I believe the emphasis on event time
>      >>> would
>      >>>> remain quite strong.
>      >>>>
>      >>>>
>      >>>> With Regards,
>      >>>>
>      >>>> Arun Mathew
>      >>>> Yahoo! JAPAN Corporation, Tokyo
>      >>>>
>      >>>>
>      >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com>
>      >> wrote:
>      >>>>
>      >>>>> Yeah I like PuncutationType much better; I just threw Time out there
>      >>>>> more as a strawman than an actual suggestion ;) I still think it's
>      >>>>> worth considering what this buys us over an additional callback. I
>      >>>>> foresee a number of punctuate implementations following this pattern:
>      >>>>>
>      >>>>> public void punctuate(PunctuationType type) {
>      >>>>>     switch (type) {
>      >>>>>         case EVENT_TIME:
>      >>>>>             methodA();
>      >>>>>             break;
>      >>>>>         case SYSTEM_TIME:
>      >>>>>             methodB();
>      >>>>>             break;
>      >>>>>     }
>      >>>>> }
>      >>>>>
>      >>>>> I guess one advantage of this approach is we could add additional
>      >>>>> punctuation types later in a backwards compatible way (like event
>      >> count
>      >>>>> as you mentioned).
>      >>>>>
>      >>>>> -Tommy
>      >>>>>
>      >>>>>
>      >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
>      >>>>>> That sounds promising.
>      >>>>>>
>      >>>>>> I am just wondering if `Time` is the best name. Maybe we want to
>      >> add
>      >>>>>> other non-time based punctuations at some point later. I would
>      >>>>>> suggest
>      >>>>>>
>      >>>>>> enum PunctuationType {
>      >>>>>>   EVENT_TIME,
>      >>>>>>   SYSTEM_TIME,
>      >>>>>> }
>      >>>>>>
>      >>>>>> or similar. Just to keep the door open -- it's easier to add new
>      >>>>>> stuff
>      >>>>>> if the name is more generic.
>      >>>>>>
>      >>>>>>
>      >>>>>> -Matthias
>      >>>>>>
>      >>>>>>
>      >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>      >>>>>>>
>      >>>>>>> I agree that the framework providing and managing the notion of
>      >>>>>>> stream
>      >>>>>>> time is valuable and not something we would want to delegate to
>      >> the
>      >>>>>>> tasks. I'm not entirely convinced that a separate callback
>      >> (option
>      >>>>>>> C)
>      >>>>>>> is that messy (it could just be a default method with an empty
>      >>>>>>> implementation), but if we wanted a single API to handle both
>      >>>>>>> cases,
>      >>>>>>> how about something like the following?
>      >>>>>>>
>      >>>>>>> enum Time {
>      >>>>>>>    STREAM,
>      >>>>>>>    CLOCK
>      >>>>>>> }
>      >>>>>>>
>      >>>>>>> Then on ProcessorContext:
>      >>>>>>> context.schedule(Time time, long interval)  // We could allow
>      >> this
>      >>>>>>> to
>      >>>>>>> be called once for each value of time to mix approaches.
>      >>>>>>>
>      >>>>>>> Then the Processor API becomes:
>      >>>>>>> punctuate(Time time) // time here denotes which schedule resulted
>      >>>>>>> in
>      >>>>>>> this call.
>      >>>>>>>
>      >>>>>>> Thoughts?
>      >>>>>>>
>      >>>>>>>
>      >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
>      >>>>>>>>
>      >>>>>>>> Thanks a lot for the KIP Michal,
>      >>>>>>>>
>      >>>>>>>> I was thinking about the four options you proposed in more
>      >>>>>>>> details
>      >>>>>>>> and
>      >>>>>>>> this are my thoughts:
>      >>>>>>>>
>      >>>>>>>> (A) You argue, that users can still "punctuate" on event-time
>      >> via
>      >>>>>>>> process(), but I am not sure if this is possible. Note, that
>      >>>>>>>> users
>      >>>>>>>> only
>      >>>>>>>> get record timestamps via context.timestamp(). Thus, users
>      >> would
>      >>>>>>>> need
>      >>>>>>>> to
>      >>>>>>>> track the time progress per partition (based on the partitions
>      >>>>>>>> they
>      >>>>>>>> obverse via context.partition(). (This alone puts a huge burden
>      >>>>>>>> on
>      >>>>>>>> the
>      >>>>>>>> user by itself.) However, users are not notified at startup
>      >> what
>      >>>>>>>> partitions are assigned, and user are not notified when
>      >>>>>>>> partitions
>      >>>>>>>> get
>      >>>>>>>> revoked. Because this information is not available, it's not
>      >>>>>>>> possible
>      >>>>>>>> to
>      >>>>>>>> "manually advance" stream-time, and thus event-time punctuation
>      >>>>>>>> within
>      >>>>>>>> process() seems not to be possible -- or do you see a way to
>      >> get
>      >>>>>>>> it
>      >>>>>>>> done? And even if, it might still be too clumsy to use.
>      >>>>>>>>
>      >>>>>>>> (B) This does not allow to mix both approaches, thus limiting
>      >>>>>>>> what
>      >>>>>>>> users
>      >>>>>>>> can do.
>      >>>>>>>>
>      >>>>>>>> (C) This should give all flexibility we need. However, just
>      >>>>>>>> adding
>      >>>>>>>> one
>      >>>>>>>> more method seems to be a solution that is too simple (cf my
>      >>>>>>>> comments
>      >>>>>>>> below).
>      >>>>>>>>
>      >>>>>>>> (D) This might be hard to use. Also, I am not sure how a user
>      >>>>>>>> could
>      >>>>>>>> enable system-time and event-time punctuation in parallel.
>      >>>>>>>>
>      >>>>>>>>
>      >>>>>>>>
>      >>>>>>>> Overall options (C) seems to be the most promising approach to
>      >>>>>>>> me.
>      >>>>>>>> Because I also favor a clean API, we might keep current
>      >>>>>>>> punctuate()
>      >>>>>>>> as-is, but deprecate it -- so we can remove it at some later
>      >>>>>>>> point
>      >>>>>>>> when
>      >>>>>>>> people use the "new punctuate API".
>      >>>>>>>>
>      >>>>>>>>
>      >>>>>>>> Couple of follow up questions:
>      >>>>>>>>
>      >>>>>>>> - I am wondering, if we should have two callback methods or
>      >> just
>      >>>>>>>> one
>      >>>>>>>> (ie, a unified for system and event time punctuation or one for
>      >>>>>>>> each?).
>      >>>>>>>>
>      >>>>>>>> - If we have one, how can the user figure out, which condition
>      >>>>>>>> did
>      >>>>>>>> trigger?
>      >>>>>>>>
>      >>>>>>>> - How would the API look like, for registering different
>      >>>>>>>> punctuate
>      >>>>>>>> schedules? The "type" must be somehow defined?
>      >>>>>>>>
>      >>>>>>>> - We might want to add "complex" schedules later on (like,
>      >>>>>>>> punctuate
>      >>>>>>>> on
>      >>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever
>      >>>>>>>> comes
>      >>>>>>>> first). I don't say we should add this right away, but we might
>      >>>>>>>> want
>      >>>>>>>> to
>      >>>>>>>> define the API in a way, that it allows extensions like this
>      >>>>>>>> later
>      >>>>>>>> on,
>      >>>>>>>> without redesigning the API (ie, the API should be designed
>      >>>>>>>> extensible)
>      >>>>>>>>
>      >>>>>>>> - Did you ever consider count-based punctuation?
>      >>>>>>>>
>      >>>>>>>>
>      >>>>>>>> I understand, that you would like to solve a simple problem,
>      >> but
>      >>>>>>>> we
>      >>>>>>>> learned from the past, that just "adding some API" quickly
>      >> leads
>      >>>>>>>> to a
>      >>>>>>>> not very well defined API that needs time consuming clean up
>      >>>>>>>> later on
>      >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
>      >>>>>>>> punctuation
>      >>>>>>>> KIP
>      >>>>>>>> with this from the beginning on to avoid later painful
>      >> redesign.
>      >>>>>>>>
>      >>>>>>>>
>      >>>>>>>>
>      >>>>>>>> -Matthias
>      >>>>>>>>
>      >>>>>>>>
>      >>>>>>>>
>      >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>      >>>>>>>>>
>      >>>>>>>>>
>      >>>>>>>>> Thanks Thomas,
>      >>>>>>>>>
>      >>>>>>>>> I'm also wary of changing the existing semantics of
>      >> punctuate,
>      >>>>>>>>> for
>      >>>>>>>>> backward compatibility reasons, although I like the
>      >> conceptual
>      >>>>>>>>> simplicity of that option.
>      >>>>>>>>>
>      >>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.
>      >> I
>      >>>>>>>>> added
>      >>>>>>>>> this to the KIP now as option (C).
>      >>>>>>>>>
>      >>>>>>>>> The TimestampExtractor mechanism is actually more flexible,
>      >> as
>      >>>>>>>>> it
>      >>>>>>>>> allows
>      >>>>>>>>> you to return any value, you're not limited to event time or
>      >>>>>>>>> system
>      >>>>>>>>> time
>      >>>>>>>>> (although I don't see an actual use case where you might need
>      >>>>>>>>> anything
>      >>>>>>>>> else then those two). Hence I also proposed the option to
>      >> allow
>      >>>>>>>>> users
>      >>>>>>>>> to, effectively, decide what "stream time" is for them given
>      >>>>>>>>> the
>      >>>>>>>>> presence or absence of messages, much like they can decide
>      >> what
>      >>>>>>>>> msg
>      >>>>>>>>> time
>      >>>>>>>>> means for them using the TimestampExtractor. What do you
>      >> think
>      >>>>>>>>> about
>      >>>>>>>>> that? This is probably most flexible but also most
>      >> complicated.
>      >>>>>>>>>
>      >>>>>>>>> All comments appreciated.
>      >>>>>>>>>
>      >>>>>>>>> Cheers,
>      >>>>>>>>>
>      >>>>>>>>> Michal
>      >>>>>>>>>
>      >>>>>>>>>
>      >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>      >>>>>>>>>>
>      >>>>>>>>>>
>      >>>>>>>>>> Although I fully agree we need a way to trigger periodic
>      >>>>>>>>>> processing
>      >>>>>>>>>> that is independent from whether and when messages arrive,
>      >>>>>>>>>> I'm
>      >>>>>>>>>> not sure
>      >>>>>>>>>> I like the idea of changing the existing semantics across
>      >> the
>      >>>>>>>>>> board.
>      >>>>>>>>>> What if we added an additional callback to Processor that
>      >> can
>      >>>>>>>>>> be
>      >>>>>>>>>> scheduled similarly to punctuate() but was always called at
>      >>>>>>>>>> fixed, wall
>      >>>>>>>>>> clock based intervals? This way you wouldn't have to give
>      >> up
>      >>>>>>>>>> the
>      >>>>>>>>>> notion
>      >>>>>>>>>> of stream time to be able to do periodic processing.
>      >>>>>>>>>>
>      >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>      >>>>>>>>>>>
>      >>>>>>>>>>>
>      >>>>>>>>>>> Hi all,
>      >>>>>>>>>>>
>      >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
>      >>>>>>>>>>> semantics
>      >>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>      >>> 138%
>      >>>>>>>>>>> 3A+C
>      >>>>>>>>>>> hange+
>      >>>>>>>>>>> punctuate+semantics>
>      >>>>>>>>>>> .
>      >>>>>>>>>>>
>      >>>>>>>>>>> Appreciating there can be different views on system-time
>      >> vs
>      >>>>>>>>>>> event-
>      >>>>>>>>>>> time
>      >>>>>>>>>>> semantics for punctuation depending on use-case and the
>      >>>>>>>>>>> importance of
>      >>>>>>>>>>> backwards compatibility of any such change, I've left it
>      >>>>>>>>>>> quite
>      >>>>>>>>>>> open
>      >>>>>>>>>>> and
>      >>>>>>>>>>> hope to fill in more info as the discussion progresses.
>      >>>>>>>>>>>
>      >>>>>>>>>>> Thanks,
>      >>>>>>>>>>> Michal
>      >>>>>>> --
>      >>>>>>>
>      >>>>>>>
>      >>>>>>>     Tommy Becker
>      >>>>>>>
>      >>>>>>>     Senior Software Engineer
>      >>>>>>>
>      >>>>>>>     O +1 919.460.4747
>      >>>>>>>
>      >>>>>>>     tivo.com
>      >>>>>>>
>      >>>>>>>
>      >>>>>>> ________________________________
>      >>>>>>>
>      >>>>>>> This email and any attachments may contain confidential and
>      >>>>>>> privileged material for the sole use of the intended recipient.
>      >> Any
>      >>>>>>> review, copying, or distribution of this email (or any
>      >> attachments)
>      >>>>>>> by others is prohibited. If you are not the intended recipient,
>      >>>>>>> please contact the sender immediately and permanently delete this
>      >>>>>>> email and any attachments. No employee or agent of TiVo Inc. is
>      >>>>>>> authorized to conclude any binding agreement on behalf of TiVo
>      >> Inc.
>      >>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a
>      >>>>>>> signed written agreement.
>      >>>>>>>
>      >>>>> --
>      >>>>>
>      >>>>>
>      >>>>>     Tommy Becker
>      >>>>>
>      >>>>>     Senior Software Engineer
>      >>>>>
>      >>>>>     O +1 919.460.4747
>      >>>>>
>      >>>>>     tivo.com
>      >>>>>
>      >>>>>
>      >>>>> ________________________________
>      >>>>>
>      >>>>> This email and any attachments may contain confidential and
>      >> privileged
>      >>>>> material for the sole use of the intended recipient. Any review,
>      >>> copying,
>      >>>>> or distribution of this email (or any attachments) by others is
>      >>>> prohibited.
>      >>>>> If you are not the intended recipient, please contact the sender
>      >>>>> immediately and permanently delete this email and any attachments. No
>      >>>>> employee or agent of TiVo Inc. is authorized to conclude any binding
>      >>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with
>      >> TiVo
>      >>>>> Inc. may only be made by a signed written agreement.
>      >>>>>
>      >>>>
>      >>>
>      >>
>      >
>      
>      
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Arun Mathew <am...@yahoo-corp.jp>.
Sure, Thanks Matthias. My id is [arunmathew88].

Of course. I was thinking of a subpage where people can collaborate. 

Will do as per Michael’s suggestion.

Regards,
Arun Mathew

On 4/7/17, 12:30, "Matthias J. Sax" <ma...@confluent.io> wrote:

    Please share your Wiki-ID and a committer can give you write access.
    
    Btw: as you did not initiate the KIP, you should not change the KIP
    without the permission of the original author -- in this case Michael.
    
    So you might also just share your thought over the mailing list and
    Michael can update the KIP page. Or, as an alternative, just create a
    subpage for the KIP page.
    
    @Michael: WDYT?
    
    
    -Matthias
    
    
    On 4/6/17 8:05 PM, Arun Mathew wrote:
    > Hi Jay,
    >           Thanks for the advise, I would like to list down the use cases as
    > per your suggestion. But it seems I don't have write permission to the
    > Apache Kafka Confluent Space. Whom shall I request for it?
    > 
    > Regarding your last question. We are using a patch in our production system
    > which does exactly this.
    > We window by the event time, but trigger punctuate in <punctuate interval>
    > duration of system time, in the absence of an event crossing the punctuate
    > event time.
    > 
    > We are using Kafka Stream for our Audit Trail, where we need to output the
    > event counts on each topic on each cluster aggregated over a 1 minute
    > window. We have to use event time to be able to cross check the counts. But
    > we need to trigger punctuate [aggregate event pushes] by system time in the
    > absence of events. Otherwise the event counts for unexpired windows would
    > be 0 which is bad.
    > 
    > "Maybe a hybrid solution works: I window by event time but trigger results
    > by system time for windows that have updated? Not really sure the details
    > of making that work. Does that work? Are there concrete examples where you
    > actually want the current behavior?"
    > 
    > --
    > With Regards,
    > 
    > Arun Mathew
    > Yahoo! JAPAN Corporation
    > 
    > On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> wrote:
    > 
    >> Hi Jay,
    >>
    >> The hybrid solution is exactly what I expect and need for our use cases
    >> when dealing with telecom data.
    >>
    >> Thanks
    >> Tianji
    >>
    >> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> wrote:
    >>
    >>> Hey guys,
    >>>
    >>> One thing I've always found super important for this kind of design work
    >> is
    >>> to do a really good job of cataloging the landscape of use cases and how
    >>> prevalent each one is. By that I mean not just listing lots of uses, but
    >>> also grouping them into categories that functionally need the same thing.
    >>> In the absence of this it is very hard to reason about design proposals.
    >>> From the proposals so far I think we have a lot of discussion around
    >>> possible apis, but less around what the user needs for different use
    >> cases
    >>> and how they would implement that using the api.
    >>>
    >>> Here is an example:
    >>> You aggregate click and impression data for a reddit like site. Every ten
    >>> minutes you want to output a ranked list of the top 10 articles ranked by
    >>> clicks/impressions for each geographical area. I want to be able run this
    >>> in steady state as well as rerun to regenerate results (or catch up if it
    >>> crashes).
    >>>
    >>> There are a couple of tricky things that seem to make this hard with
    >> either
    >>> of the options proposed:
    >>> 1. If I emit this data using event time I have the problem described
    >> where
    >>> a geographical region with no new clicks or impressions will fail to
    >> output
    >>> results.
    >>> 2. If I emit this data using system time I have the problem that when
    >>> reprocessing data my window may not be ten minutes but 10 hours if my
    >>> processing is very fast so it dramatically changes the output.
    >>>
    >>> Maybe a hybrid solution works: I window by event time but trigger results
    >>> by system time for windows that have updated? Not really sure the details
    >>> of making that work. Does that work? Are there concrete examples where
    >> you
    >>> actually want the current behavior?
    >>>
    >>> -Jay
    >>>
    >>>
    >>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com>
    >>> wrote:
    >>>
    >>>> Hi All,
    >>>>
    >>>> Thanks for the KIP. We were also in need of a mechanism to trigger
    >>>> punctuate in the absence of events.
    >>>>
    >>>> As I described in [
    >>>> https://issues.apache.org/jira/browse/KAFKA-3514?
    >>>> focusedCommentId=15926036&page=com.atlassian.jira.
    >>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
    >>>> ],
    >>>>
    >>>>    - Our approached involved using the event time by default.
    >>>>    - The method to check if there is any punctuate ready in the
    >>>>    PunctuationQueue is triggered via the any event received by the
    >> stream
    >>>>    tread, or at the polling intervals in the absence of any events.
    >>>>    - When we create Punctuate objects (which contains the next event
    >> time
    >>>>    for punctuation and interval), we also record the creation time
    >>> (system
    >>>>    time).
    >>>>    - While checking for maturity of Punctuate Schedule by
    >> mayBePunctuate
    >>>>    method, we also check if the system clock has elapsed the punctuate
    >>>>    interval since the schedule creation time.
    >>>>    - In the absence of any event, or in the absence of any event for
    >> one
    >>>>    topic in the partition group assigned to the stream task, the system
    >>>> time
    >>>>    will elapse the interval and we trigger a punctuate using the
    >> expected
    >>>>    punctuation event time.
    >>>>    - we then create the next punctuation schedule as punctuation event
    >>> time
    >>>>    + punctuation interval, [again recording the system time of creation
    >>> of
    >>>> the
    >>>>    schedule].
    >>>>
    >>>> We call this a Hybrid Punctuate. Of course, this approach has pros and
    >>>> cons.
    >>>> Pros
    >>>>
    >>>>    - Punctuates will happen in <punctuate interval> time duration at
    >> max
    >>> in
    >>>>    terms of system time.
    >>>>    - The semantics as a whole continues to revolve around event time.
    >>>>    - We can use the old data [old timestamps] to rerun any experiments
    >> or
    >>>>    tests.
    >>>>
    >>>> Cons
    >>>>
    >>>>    - In case the  <punctuate interval> is not a time duration [say
    >>> logical
    >>>>    time/event count], then the approach might not be meaningful.
    >>>>    - In case there is a case where we have to wait for an actual event
    >>> from
    >>>>    a low event rate partition in the partition group, this approach
    >> will
    >>>> jump
    >>>>    the gun.
    >>>>    - in case the event processing cannot catch up with the event rate
    >> and
    >>>>    the expected timestamp events gets queued for long time, this
    >> approach
    >>>>    might jump the gun.
    >>>>
    >>>> I believe the above approach and discussion goes close to the approach
    >> A.
    >>>>
    >>>> -----------
    >>>>
    >>>> I like the idea of having an even count based punctuate.
    >>>>
    >>>> -----------
    >>>>
    >>>> I agree with the discussion around approach C, that we should provide
    >> the
    >>>> user with the option to choose system time or event time based
    >>> punctuates.
    >>>> But I believe that the user predominantly wants to use event time while
    >>> not
    >>>> missing out on regular punctuates due to event delays or event
    >> absences.
    >>>> Hence a complex punctuate option as Matthias mentioned (quoted below)
    >>> would
    >>>> be most apt.
    >>>>
    >>>> "- We might want to add "complex" schedules later on (like, punctuate
    >> on
    >>>> every 10 seconds event-time or 60 seconds system-time whatever comes
    >>>> first)."
    >>>>
    >>>> -----------
    >>>>
    >>>> I think I read somewhere that Kafka Streams started with System Time as
    >>> the
    >>>> punctuation standard, but was later changed to Event Time. I guess
    >> there
    >>>> would be some good reason behind it. As Kafka Streams want to evolve
    >> more
    >>>> on the Stream Processing front, I believe the emphasis on event time
    >>> would
    >>>> remain quite strong.
    >>>>
    >>>>
    >>>> With Regards,
    >>>>
    >>>> Arun Mathew
    >>>> Yahoo! JAPAN Corporation, Tokyo
    >>>>
    >>>>
    >>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com>
    >> wrote:
    >>>>
    >>>>> Yeah I like PuncutationType much better; I just threw Time out there
    >>>>> more as a strawman than an actual suggestion ;) I still think it's
    >>>>> worth considering what this buys us over an additional callback. I
    >>>>> foresee a number of punctuate implementations following this pattern:
    >>>>>
    >>>>> public void punctuate(PunctuationType type) {
    >>>>>     switch (type) {
    >>>>>         case EVENT_TIME:
    >>>>>             methodA();
    >>>>>             break;
    >>>>>         case SYSTEM_TIME:
    >>>>>             methodB();
    >>>>>             break;
    >>>>>     }
    >>>>> }
    >>>>>
    >>>>> I guess one advantage of this approach is we could add additional
    >>>>> punctuation types later in a backwards compatible way (like event
    >> count
    >>>>> as you mentioned).
    >>>>>
    >>>>> -Tommy
    >>>>>
    >>>>>
    >>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
    >>>>>> That sounds promising.
    >>>>>>
    >>>>>> I am just wondering if `Time` is the best name. Maybe we want to
    >> add
    >>>>>> other non-time based punctuations at some point later. I would
    >>>>>> suggest
    >>>>>>
    >>>>>> enum PunctuationType {
    >>>>>>   EVENT_TIME,
    >>>>>>   SYSTEM_TIME,
    >>>>>> }
    >>>>>>
    >>>>>> or similar. Just to keep the door open -- it's easier to add new
    >>>>>> stuff
    >>>>>> if the name is more generic.
    >>>>>>
    >>>>>>
    >>>>>> -Matthias
    >>>>>>
    >>>>>>
    >>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
    >>>>>>>
    >>>>>>> I agree that the framework providing and managing the notion of
    >>>>>>> stream
    >>>>>>> time is valuable and not something we would want to delegate to
    >> the
    >>>>>>> tasks. I'm not entirely convinced that a separate callback
    >> (option
    >>>>>>> C)
    >>>>>>> is that messy (it could just be a default method with an empty
    >>>>>>> implementation), but if we wanted a single API to handle both
    >>>>>>> cases,
    >>>>>>> how about something like the following?
    >>>>>>>
    >>>>>>> enum Time {
    >>>>>>>    STREAM,
    >>>>>>>    CLOCK
    >>>>>>> }
    >>>>>>>
    >>>>>>> Then on ProcessorContext:
    >>>>>>> context.schedule(Time time, long interval)  // We could allow
    >> this
    >>>>>>> to
    >>>>>>> be called once for each value of time to mix approaches.
    >>>>>>>
    >>>>>>> Then the Processor API becomes:
    >>>>>>> punctuate(Time time) // time here denotes which schedule resulted
    >>>>>>> in
    >>>>>>> this call.
    >>>>>>>
    >>>>>>> Thoughts?
    >>>>>>>
    >>>>>>>
    >>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
    >>>>>>>>
    >>>>>>>> Thanks a lot for the KIP Michal,
    >>>>>>>>
    >>>>>>>> I was thinking about the four options you proposed in more
    >>>>>>>> details
    >>>>>>>> and
    >>>>>>>> this are my thoughts:
    >>>>>>>>
    >>>>>>>> (A) You argue, that users can still "punctuate" on event-time
    >> via
    >>>>>>>> process(), but I am not sure if this is possible. Note, that
    >>>>>>>> users
    >>>>>>>> only
    >>>>>>>> get record timestamps via context.timestamp(). Thus, users
    >> would
    >>>>>>>> need
    >>>>>>>> to
    >>>>>>>> track the time progress per partition (based on the partitions
    >>>>>>>> they
    >>>>>>>> obverse via context.partition(). (This alone puts a huge burden
    >>>>>>>> on
    >>>>>>>> the
    >>>>>>>> user by itself.) However, users are not notified at startup
    >> what
    >>>>>>>> partitions are assigned, and user are not notified when
    >>>>>>>> partitions
    >>>>>>>> get
    >>>>>>>> revoked. Because this information is not available, it's not
    >>>>>>>> possible
    >>>>>>>> to
    >>>>>>>> "manually advance" stream-time, and thus event-time punctuation
    >>>>>>>> within
    >>>>>>>> process() seems not to be possible -- or do you see a way to
    >> get
    >>>>>>>> it
    >>>>>>>> done? And even if, it might still be too clumsy to use.
    >>>>>>>>
    >>>>>>>> (B) This does not allow to mix both approaches, thus limiting
    >>>>>>>> what
    >>>>>>>> users
    >>>>>>>> can do.
    >>>>>>>>
    >>>>>>>> (C) This should give all flexibility we need. However, just
    >>>>>>>> adding
    >>>>>>>> one
    >>>>>>>> more method seems to be a solution that is too simple (cf my
    >>>>>>>> comments
    >>>>>>>> below).
    >>>>>>>>
    >>>>>>>> (D) This might be hard to use. Also, I am not sure how a user
    >>>>>>>> could
    >>>>>>>> enable system-time and event-time punctuation in parallel.
    >>>>>>>>
    >>>>>>>>
    >>>>>>>>
    >>>>>>>> Overall options (C) seems to be the most promising approach to
    >>>>>>>> me.
    >>>>>>>> Because I also favor a clean API, we might keep current
    >>>>>>>> punctuate()
    >>>>>>>> as-is, but deprecate it -- so we can remove it at some later
    >>>>>>>> point
    >>>>>>>> when
    >>>>>>>> people use the "new punctuate API".
    >>>>>>>>
    >>>>>>>>
    >>>>>>>> Couple of follow up questions:
    >>>>>>>>
    >>>>>>>> - I am wondering, if we should have two callback methods or
    >> just
    >>>>>>>> one
    >>>>>>>> (ie, a unified for system and event time punctuation or one for
    >>>>>>>> each?).
    >>>>>>>>
    >>>>>>>> - If we have one, how can the user figure out, which condition
    >>>>>>>> did
    >>>>>>>> trigger?
    >>>>>>>>
    >>>>>>>> - How would the API look like, for registering different
    >>>>>>>> punctuate
    >>>>>>>> schedules? The "type" must be somehow defined?
    >>>>>>>>
    >>>>>>>> - We might want to add "complex" schedules later on (like,
    >>>>>>>> punctuate
    >>>>>>>> on
    >>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever
    >>>>>>>> comes
    >>>>>>>> first). I don't say we should add this right away, but we might
    >>>>>>>> want
    >>>>>>>> to
    >>>>>>>> define the API in a way, that it allows extensions like this
    >>>>>>>> later
    >>>>>>>> on,
    >>>>>>>> without redesigning the API (ie, the API should be designed
    >>>>>>>> extensible)
    >>>>>>>>
    >>>>>>>> - Did you ever consider count-based punctuation?
    >>>>>>>>
    >>>>>>>>
    >>>>>>>> I understand, that you would like to solve a simple problem,
    >> but
    >>>>>>>> we
    >>>>>>>> learned from the past, that just "adding some API" quickly
    >> leads
    >>>>>>>> to a
    >>>>>>>> not very well defined API that needs time consuming clean up
    >>>>>>>> later on
    >>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
    >>>>>>>> punctuation
    >>>>>>>> KIP
    >>>>>>>> with this from the beginning on to avoid later painful
    >> redesign.
    >>>>>>>>
    >>>>>>>>
    >>>>>>>>
    >>>>>>>> -Matthias
    >>>>>>>>
    >>>>>>>>
    >>>>>>>>
    >>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
    >>>>>>>>>
    >>>>>>>>>
    >>>>>>>>> Thanks Thomas,
    >>>>>>>>>
    >>>>>>>>> I'm also wary of changing the existing semantics of
    >> punctuate,
    >>>>>>>>> for
    >>>>>>>>> backward compatibility reasons, although I like the
    >> conceptual
    >>>>>>>>> simplicity of that option.
    >>>>>>>>>
    >>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.
    >> I
    >>>>>>>>> added
    >>>>>>>>> this to the KIP now as option (C).
    >>>>>>>>>
    >>>>>>>>> The TimestampExtractor mechanism is actually more flexible,
    >> as
    >>>>>>>>> it
    >>>>>>>>> allows
    >>>>>>>>> you to return any value, you're not limited to event time or
    >>>>>>>>> system
    >>>>>>>>> time
    >>>>>>>>> (although I don't see an actual use case where you might need
    >>>>>>>>> anything
    >>>>>>>>> else then those two). Hence I also proposed the option to
    >> allow
    >>>>>>>>> users
    >>>>>>>>> to, effectively, decide what "stream time" is for them given
    >>>>>>>>> the
    >>>>>>>>> presence or absence of messages, much like they can decide
    >> what
    >>>>>>>>> msg
    >>>>>>>>> time
    >>>>>>>>> means for them using the TimestampExtractor. What do you
    >> think
    >>>>>>>>> about
    >>>>>>>>> that? This is probably most flexible but also most
    >> complicated.
    >>>>>>>>>
    >>>>>>>>> All comments appreciated.
    >>>>>>>>>
    >>>>>>>>> Cheers,
    >>>>>>>>>
    >>>>>>>>> Michal
    >>>>>>>>>
    >>>>>>>>>
    >>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
    >>>>>>>>>>
    >>>>>>>>>>
    >>>>>>>>>> Although I fully agree we need a way to trigger periodic
    >>>>>>>>>> processing
    >>>>>>>>>> that is independent from whether and when messages arrive,
    >>>>>>>>>> I'm
    >>>>>>>>>> not sure
    >>>>>>>>>> I like the idea of changing the existing semantics across
    >> the
    >>>>>>>>>> board.
    >>>>>>>>>> What if we added an additional callback to Processor that
    >> can
    >>>>>>>>>> be
    >>>>>>>>>> scheduled similarly to punctuate() but was always called at
    >>>>>>>>>> fixed, wall
    >>>>>>>>>> clock based intervals? This way you wouldn't have to give
    >> up
    >>>>>>>>>> the
    >>>>>>>>>> notion
    >>>>>>>>>> of stream time to be able to do periodic processing.
    >>>>>>>>>>
    >>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
    >>>>>>>>>>>
    >>>>>>>>>>>
    >>>>>>>>>>> Hi all,
    >>>>>>>>>>>
    >>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
    >>>>>>>>>>> semantics
    >>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
    >>> 138%
    >>>>>>>>>>> 3A+C
    >>>>>>>>>>> hange+
    >>>>>>>>>>> punctuate+semantics>
    >>>>>>>>>>> .
    >>>>>>>>>>>
    >>>>>>>>>>> Appreciating there can be different views on system-time
    >> vs
    >>>>>>>>>>> event-
    >>>>>>>>>>> time
    >>>>>>>>>>> semantics for punctuation depending on use-case and the
    >>>>>>>>>>> importance of
    >>>>>>>>>>> backwards compatibility of any such change, I've left it
    >>>>>>>>>>> quite
    >>>>>>>>>>> open
    >>>>>>>>>>> and
    >>>>>>>>>>> hope to fill in more info as the discussion progresses.
    >>>>>>>>>>>
    >>>>>>>>>>> Thanks,
    >>>>>>>>>>> Michal
    >>>>>>> --
    >>>>>>>
    >>>>>>>
    >>>>>>>     Tommy Becker
    >>>>>>>
    >>>>>>>     Senior Software Engineer
    >>>>>>>
    >>>>>>>     O +1 919.460.4747
    >>>>>>>
    >>>>>>>     tivo.com
    >>>>>>>
    >>>>>>>
    >>>>>>> ________________________________
    >>>>>>>
    >>>>>>> This email and any attachments may contain confidential and
    >>>>>>> privileged material for the sole use of the intended recipient.
    >> Any
    >>>>>>> review, copying, or distribution of this email (or any
    >> attachments)
    >>>>>>> by others is prohibited. If you are not the intended recipient,
    >>>>>>> please contact the sender immediately and permanently delete this
    >>>>>>> email and any attachments. No employee or agent of TiVo Inc. is
    >>>>>>> authorized to conclude any binding agreement on behalf of TiVo
    >> Inc.
    >>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a
    >>>>>>> signed written agreement.
    >>>>>>>
    >>>>> --
    >>>>>
    >>>>>
    >>>>>     Tommy Becker
    >>>>>
    >>>>>     Senior Software Engineer
    >>>>>
    >>>>>     O +1 919.460.4747
    >>>>>
    >>>>>     tivo.com
    >>>>>
    >>>>>
    >>>>> ________________________________
    >>>>>
    >>>>> This email and any attachments may contain confidential and
    >> privileged
    >>>>> material for the sole use of the intended recipient. Any review,
    >>> copying,
    >>>>> or distribution of this email (or any attachments) by others is
    >>>> prohibited.
    >>>>> If you are not the intended recipient, please contact the sender
    >>>>> immediately and permanently delete this email and any attachments. No
    >>>>> employee or agent of TiVo Inc. is authorized to conclude any binding
    >>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with
    >> TiVo
    >>>>> Inc. may only be made by a signed written agreement.
    >>>>>
    >>>>
    >>>
    >>
    > 
    
    


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Please share your Wiki-ID and a committer can give you write access.

Btw: as you did not initiate the KIP, you should not change the KIP
without the permission of the original author -- in this case Michael.

So you might also just share your thought over the mailing list and
Michael can update the KIP page. Or, as an alternative, just create a
subpage for the KIP page.

@Michael: WDYT?


-Matthias


On 4/6/17 8:05 PM, Arun Mathew wrote:
> Hi Jay,
>           Thanks for the advise, I would like to list down the use cases as
> per your suggestion. But it seems I don't have write permission to the
> Apache Kafka Confluent Space. Whom shall I request for it?
> 
> Regarding your last question. We are using a patch in our production system
> which does exactly this.
> We window by the event time, but trigger punctuate in <punctuate interval>
> duration of system time, in the absence of an event crossing the punctuate
> event time.
> 
> We are using Kafka Stream for our Audit Trail, where we need to output the
> event counts on each topic on each cluster aggregated over a 1 minute
> window. We have to use event time to be able to cross check the counts. But
> we need to trigger punctuate [aggregate event pushes] by system time in the
> absence of events. Otherwise the event counts for unexpired windows would
> be 0 which is bad.
> 
> "Maybe a hybrid solution works: I window by event time but trigger results
> by system time for windows that have updated? Not really sure the details
> of making that work. Does that work? Are there concrete examples where you
> actually want the current behavior?"
> 
> --
> With Regards,
> 
> Arun Mathew
> Yahoo! JAPAN Corporation
> 
> On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> wrote:
> 
>> Hi Jay,
>>
>> The hybrid solution is exactly what I expect and need for our use cases
>> when dealing with telecom data.
>>
>> Thanks
>> Tianji
>>
>> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> wrote:
>>
>>> Hey guys,
>>>
>>> One thing I've always found super important for this kind of design work
>> is
>>> to do a really good job of cataloging the landscape of use cases and how
>>> prevalent each one is. By that I mean not just listing lots of uses, but
>>> also grouping them into categories that functionally need the same thing.
>>> In the absence of this it is very hard to reason about design proposals.
>>> From the proposals so far I think we have a lot of discussion around
>>> possible apis, but less around what the user needs for different use
>> cases
>>> and how they would implement that using the api.
>>>
>>> Here is an example:
>>> You aggregate click and impression data for a reddit like site. Every ten
>>> minutes you want to output a ranked list of the top 10 articles ranked by
>>> clicks/impressions for each geographical area. I want to be able run this
>>> in steady state as well as rerun to regenerate results (or catch up if it
>>> crashes).
>>>
>>> There are a couple of tricky things that seem to make this hard with
>> either
>>> of the options proposed:
>>> 1. If I emit this data using event time I have the problem described
>> where
>>> a geographical region with no new clicks or impressions will fail to
>> output
>>> results.
>>> 2. If I emit this data using system time I have the problem that when
>>> reprocessing data my window may not be ten minutes but 10 hours if my
>>> processing is very fast so it dramatically changes the output.
>>>
>>> Maybe a hybrid solution works: I window by event time but trigger results
>>> by system time for windows that have updated? Not really sure the details
>>> of making that work. Does that work? Are there concrete examples where
>> you
>>> actually want the current behavior?
>>>
>>> -Jay
>>>
>>>
>>> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com>
>>> wrote:
>>>
>>>> Hi All,
>>>>
>>>> Thanks for the KIP. We were also in need of a mechanism to trigger
>>>> punctuate in the absence of events.
>>>>
>>>> As I described in [
>>>> https://issues.apache.org/jira/browse/KAFKA-3514?
>>>> focusedCommentId=15926036&page=com.atlassian.jira.
>>>> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
>>>> ],
>>>>
>>>>    - Our approached involved using the event time by default.
>>>>    - The method to check if there is any punctuate ready in the
>>>>    PunctuationQueue is triggered via the any event received by the
>> stream
>>>>    tread, or at the polling intervals in the absence of any events.
>>>>    - When we create Punctuate objects (which contains the next event
>> time
>>>>    for punctuation and interval), we also record the creation time
>>> (system
>>>>    time).
>>>>    - While checking for maturity of Punctuate Schedule by
>> mayBePunctuate
>>>>    method, we also check if the system clock has elapsed the punctuate
>>>>    interval since the schedule creation time.
>>>>    - In the absence of any event, or in the absence of any event for
>> one
>>>>    topic in the partition group assigned to the stream task, the system
>>>> time
>>>>    will elapse the interval and we trigger a punctuate using the
>> expected
>>>>    punctuation event time.
>>>>    - we then create the next punctuation schedule as punctuation event
>>> time
>>>>    + punctuation interval, [again recording the system time of creation
>>> of
>>>> the
>>>>    schedule].
>>>>
>>>> We call this a Hybrid Punctuate. Of course, this approach has pros and
>>>> cons.
>>>> Pros
>>>>
>>>>    - Punctuates will happen in <punctuate interval> time duration at
>> max
>>> in
>>>>    terms of system time.
>>>>    - The semantics as a whole continues to revolve around event time.
>>>>    - We can use the old data [old timestamps] to rerun any experiments
>> or
>>>>    tests.
>>>>
>>>> Cons
>>>>
>>>>    - In case the  <punctuate interval> is not a time duration [say
>>> logical
>>>>    time/event count], then the approach might not be meaningful.
>>>>    - In case there is a case where we have to wait for an actual event
>>> from
>>>>    a low event rate partition in the partition group, this approach
>> will
>>>> jump
>>>>    the gun.
>>>>    - in case the event processing cannot catch up with the event rate
>> and
>>>>    the expected timestamp events gets queued for long time, this
>> approach
>>>>    might jump the gun.
>>>>
>>>> I believe the above approach and discussion goes close to the approach
>> A.
>>>>
>>>> -----------
>>>>
>>>> I like the idea of having an even count based punctuate.
>>>>
>>>> -----------
>>>>
>>>> I agree with the discussion around approach C, that we should provide
>> the
>>>> user with the option to choose system time or event time based
>>> punctuates.
>>>> But I believe that the user predominantly wants to use event time while
>>> not
>>>> missing out on regular punctuates due to event delays or event
>> absences.
>>>> Hence a complex punctuate option as Matthias mentioned (quoted below)
>>> would
>>>> be most apt.
>>>>
>>>> "- We might want to add "complex" schedules later on (like, punctuate
>> on
>>>> every 10 seconds event-time or 60 seconds system-time whatever comes
>>>> first)."
>>>>
>>>> -----------
>>>>
>>>> I think I read somewhere that Kafka Streams started with System Time as
>>> the
>>>> punctuation standard, but was later changed to Event Time. I guess
>> there
>>>> would be some good reason behind it. As Kafka Streams want to evolve
>> more
>>>> on the Stream Processing front, I believe the emphasis on event time
>>> would
>>>> remain quite strong.
>>>>
>>>>
>>>> With Regards,
>>>>
>>>> Arun Mathew
>>>> Yahoo! JAPAN Corporation, Tokyo
>>>>
>>>>
>>>> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com>
>> wrote:
>>>>
>>>>> Yeah I like PuncutationType much better; I just threw Time out there
>>>>> more as a strawman than an actual suggestion ;) I still think it's
>>>>> worth considering what this buys us over an additional callback. I
>>>>> foresee a number of punctuate implementations following this pattern:
>>>>>
>>>>> public void punctuate(PunctuationType type) {
>>>>>     switch (type) {
>>>>>         case EVENT_TIME:
>>>>>             methodA();
>>>>>             break;
>>>>>         case SYSTEM_TIME:
>>>>>             methodB();
>>>>>             break;
>>>>>     }
>>>>> }
>>>>>
>>>>> I guess one advantage of this approach is we could add additional
>>>>> punctuation types later in a backwards compatible way (like event
>> count
>>>>> as you mentioned).
>>>>>
>>>>> -Tommy
>>>>>
>>>>>
>>>>> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
>>>>>> That sounds promising.
>>>>>>
>>>>>> I am just wondering if `Time` is the best name. Maybe we want to
>> add
>>>>>> other non-time based punctuations at some point later. I would
>>>>>> suggest
>>>>>>
>>>>>> enum PunctuationType {
>>>>>>   EVENT_TIME,
>>>>>>   SYSTEM_TIME,
>>>>>> }
>>>>>>
>>>>>> or similar. Just to keep the door open -- it's easier to add new
>>>>>> stuff
>>>>>> if the name is more generic.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>> On 4/4/17 5:30 AM, Thomas Becker wrote:
>>>>>>>
>>>>>>> I agree that the framework providing and managing the notion of
>>>>>>> stream
>>>>>>> time is valuable and not something we would want to delegate to
>> the
>>>>>>> tasks. I'm not entirely convinced that a separate callback
>> (option
>>>>>>> C)
>>>>>>> is that messy (it could just be a default method with an empty
>>>>>>> implementation), but if we wanted a single API to handle both
>>>>>>> cases,
>>>>>>> how about something like the following?
>>>>>>>
>>>>>>> enum Time {
>>>>>>>    STREAM,
>>>>>>>    CLOCK
>>>>>>> }
>>>>>>>
>>>>>>> Then on ProcessorContext:
>>>>>>> context.schedule(Time time, long interval)  // We could allow
>> this
>>>>>>> to
>>>>>>> be called once for each value of time to mix approaches.
>>>>>>>
>>>>>>> Then the Processor API becomes:
>>>>>>> punctuate(Time time) // time here denotes which schedule resulted
>>>>>>> in
>>>>>>> this call.
>>>>>>>
>>>>>>> Thoughts?
>>>>>>>
>>>>>>>
>>>>>>> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
>>>>>>>>
>>>>>>>> Thanks a lot for the KIP Michal,
>>>>>>>>
>>>>>>>> I was thinking about the four options you proposed in more
>>>>>>>> details
>>>>>>>> and
>>>>>>>> this are my thoughts:
>>>>>>>>
>>>>>>>> (A) You argue, that users can still "punctuate" on event-time
>> via
>>>>>>>> process(), but I am not sure if this is possible. Note, that
>>>>>>>> users
>>>>>>>> only
>>>>>>>> get record timestamps via context.timestamp(). Thus, users
>> would
>>>>>>>> need
>>>>>>>> to
>>>>>>>> track the time progress per partition (based on the partitions
>>>>>>>> they
>>>>>>>> obverse via context.partition(). (This alone puts a huge burden
>>>>>>>> on
>>>>>>>> the
>>>>>>>> user by itself.) However, users are not notified at startup
>> what
>>>>>>>> partitions are assigned, and user are not notified when
>>>>>>>> partitions
>>>>>>>> get
>>>>>>>> revoked. Because this information is not available, it's not
>>>>>>>> possible
>>>>>>>> to
>>>>>>>> "manually advance" stream-time, and thus event-time punctuation
>>>>>>>> within
>>>>>>>> process() seems not to be possible -- or do you see a way to
>> get
>>>>>>>> it
>>>>>>>> done? And even if, it might still be too clumsy to use.
>>>>>>>>
>>>>>>>> (B) This does not allow to mix both approaches, thus limiting
>>>>>>>> what
>>>>>>>> users
>>>>>>>> can do.
>>>>>>>>
>>>>>>>> (C) This should give all flexibility we need. However, just
>>>>>>>> adding
>>>>>>>> one
>>>>>>>> more method seems to be a solution that is too simple (cf my
>>>>>>>> comments
>>>>>>>> below).
>>>>>>>>
>>>>>>>> (D) This might be hard to use. Also, I am not sure how a user
>>>>>>>> could
>>>>>>>> enable system-time and event-time punctuation in parallel.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> Overall options (C) seems to be the most promising approach to
>>>>>>>> me.
>>>>>>>> Because I also favor a clean API, we might keep current
>>>>>>>> punctuate()
>>>>>>>> as-is, but deprecate it -- so we can remove it at some later
>>>>>>>> point
>>>>>>>> when
>>>>>>>> people use the "new punctuate API".
>>>>>>>>
>>>>>>>>
>>>>>>>> Couple of follow up questions:
>>>>>>>>
>>>>>>>> - I am wondering, if we should have two callback methods or
>> just
>>>>>>>> one
>>>>>>>> (ie, a unified for system and event time punctuation or one for
>>>>>>>> each?).
>>>>>>>>
>>>>>>>> - If we have one, how can the user figure out, which condition
>>>>>>>> did
>>>>>>>> trigger?
>>>>>>>>
>>>>>>>> - How would the API look like, for registering different
>>>>>>>> punctuate
>>>>>>>> schedules? The "type" must be somehow defined?
>>>>>>>>
>>>>>>>> - We might want to add "complex" schedules later on (like,
>>>>>>>> punctuate
>>>>>>>> on
>>>>>>>> every 10 seconds event-time or 60 seconds system-time whatever
>>>>>>>> comes
>>>>>>>> first). I don't say we should add this right away, but we might
>>>>>>>> want
>>>>>>>> to
>>>>>>>> define the API in a way, that it allows extensions like this
>>>>>>>> later
>>>>>>>> on,
>>>>>>>> without redesigning the API (ie, the API should be designed
>>>>>>>> extensible)
>>>>>>>>
>>>>>>>> - Did you ever consider count-based punctuation?
>>>>>>>>
>>>>>>>>
>>>>>>>> I understand, that you would like to solve a simple problem,
>> but
>>>>>>>> we
>>>>>>>> learned from the past, that just "adding some API" quickly
>> leads
>>>>>>>> to a
>>>>>>>> not very well defined API that needs time consuming clean up
>>>>>>>> later on
>>>>>>>> via other KIPs. Thus, I would prefer to get a holistic
>>>>>>>> punctuation
>>>>>>>> KIP
>>>>>>>> with this from the beginning on to avoid later painful
>> redesign.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Thanks Thomas,
>>>>>>>>>
>>>>>>>>> I'm also wary of changing the existing semantics of
>> punctuate,
>>>>>>>>> for
>>>>>>>>> backward compatibility reasons, although I like the
>> conceptual
>>>>>>>>> simplicity of that option.
>>>>>>>>>
>>>>>>>>> Adding a new method to me feels safer but, in a way, uglier.
>> I
>>>>>>>>> added
>>>>>>>>> this to the KIP now as option (C).
>>>>>>>>>
>>>>>>>>> The TimestampExtractor mechanism is actually more flexible,
>> as
>>>>>>>>> it
>>>>>>>>> allows
>>>>>>>>> you to return any value, you're not limited to event time or
>>>>>>>>> system
>>>>>>>>> time
>>>>>>>>> (although I don't see an actual use case where you might need
>>>>>>>>> anything
>>>>>>>>> else then those two). Hence I also proposed the option to
>> allow
>>>>>>>>> users
>>>>>>>>> to, effectively, decide what "stream time" is for them given
>>>>>>>>> the
>>>>>>>>> presence or absence of messages, much like they can decide
>> what
>>>>>>>>> msg
>>>>>>>>> time
>>>>>>>>> means for them using the TimestampExtractor. What do you
>> think
>>>>>>>>> about
>>>>>>>>> that? This is probably most flexible but also most
>> complicated.
>>>>>>>>>
>>>>>>>>> All comments appreciated.
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>>
>>>>>>>>> Michal
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Although I fully agree we need a way to trigger periodic
>>>>>>>>>> processing
>>>>>>>>>> that is independent from whether and when messages arrive,
>>>>>>>>>> I'm
>>>>>>>>>> not sure
>>>>>>>>>> I like the idea of changing the existing semantics across
>> the
>>>>>>>>>> board.
>>>>>>>>>> What if we added an additional callback to Processor that
>> can
>>>>>>>>>> be
>>>>>>>>>> scheduled similarly to punctuate() but was always called at
>>>>>>>>>> fixed, wall
>>>>>>>>>> clock based intervals? This way you wouldn't have to give
>> up
>>>>>>>>>> the
>>>>>>>>>> notion
>>>>>>>>>> of stream time to be able to do periodic processing.
>>>>>>>>>>
>>>>>>>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Hi all,
>>>>>>>>>>>
>>>>>>>>>>> I have created a draft for KIP-138: Change punctuate
>>>>>>>>>>> semantics
>>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 138%
>>>>>>>>>>> 3A+C
>>>>>>>>>>> hange+
>>>>>>>>>>> punctuate+semantics>
>>>>>>>>>>> .
>>>>>>>>>>>
>>>>>>>>>>> Appreciating there can be different views on system-time
>> vs
>>>>>>>>>>> event-
>>>>>>>>>>> time
>>>>>>>>>>> semantics for punctuation depending on use-case and the
>>>>>>>>>>> importance of
>>>>>>>>>>> backwards compatibility of any such change, I've left it
>>>>>>>>>>> quite
>>>>>>>>>>> open
>>>>>>>>>>> and
>>>>>>>>>>> hope to fill in more info as the discussion progresses.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Michal
>>>>>>> --
>>>>>>>
>>>>>>>
>>>>>>>     Tommy Becker
>>>>>>>
>>>>>>>     Senior Software Engineer
>>>>>>>
>>>>>>>     O +1 919.460.4747
>>>>>>>
>>>>>>>     tivo.com
>>>>>>>
>>>>>>>
>>>>>>> ________________________________
>>>>>>>
>>>>>>> This email and any attachments may contain confidential and
>>>>>>> privileged material for the sole use of the intended recipient.
>> Any
>>>>>>> review, copying, or distribution of this email (or any
>> attachments)
>>>>>>> by others is prohibited. If you are not the intended recipient,
>>>>>>> please contact the sender immediately and permanently delete this
>>>>>>> email and any attachments. No employee or agent of TiVo Inc. is
>>>>>>> authorized to conclude any binding agreement on behalf of TiVo
>> Inc.
>>>>>>> by email. Binding agreements with TiVo Inc. may only be made by a
>>>>>>> signed written agreement.
>>>>>>>
>>>>> --
>>>>>
>>>>>
>>>>>     Tommy Becker
>>>>>
>>>>>     Senior Software Engineer
>>>>>
>>>>>     O +1 919.460.4747
>>>>>
>>>>>     tivo.com
>>>>>
>>>>>
>>>>> ________________________________
>>>>>
>>>>> This email and any attachments may contain confidential and
>> privileged
>>>>> material for the sole use of the intended recipient. Any review,
>>> copying,
>>>>> or distribution of this email (or any attachments) by others is
>>>> prohibited.
>>>>> If you are not the intended recipient, please contact the sender
>>>>> immediately and permanently delete this email and any attachments. No
>>>>> employee or agent of TiVo Inc. is authorized to conclude any binding
>>>>> agreement on behalf of TiVo Inc. by email. Binding agreements with
>> TiVo
>>>>> Inc. may only be made by a signed written agreement.
>>>>>
>>>>
>>>
>>
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Arun Mathew <ar...@gmail.com>.
Hi Jay,
          Thanks for the advise, I would like to list down the use cases as
per your suggestion. But it seems I don't have write permission to the
Apache Kafka Confluent Space. Whom shall I request for it?

Regarding your last question. We are using a patch in our production system
which does exactly this.
We window by the event time, but trigger punctuate in <punctuate interval>
duration of system time, in the absence of an event crossing the punctuate
event time.

We are using Kafka Stream for our Audit Trail, where we need to output the
event counts on each topic on each cluster aggregated over a 1 minute
window. We have to use event time to be able to cross check the counts. But
we need to trigger punctuate [aggregate event pushes] by system time in the
absence of events. Otherwise the event counts for unexpired windows would
be 0 which is bad.

"Maybe a hybrid solution works: I window by event time but trigger results
by system time for windows that have updated? Not really sure the details
of making that work. Does that work? Are there concrete examples where you
actually want the current behavior?"

--
With Regards,

Arun Mathew
Yahoo! JAPAN Corporation

On Wed, Apr 5, 2017 at 8:48 PM, Tianji Li <sk...@gmail.com> wrote:

> Hi Jay,
>
> The hybrid solution is exactly what I expect and need for our use cases
> when dealing with telecom data.
>
> Thanks
> Tianji
>
> On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey guys,
> >
> > One thing I've always found super important for this kind of design work
> is
> > to do a really good job of cataloging the landscape of use cases and how
> > prevalent each one is. By that I mean not just listing lots of uses, but
> > also grouping them into categories that functionally need the same thing.
> > In the absence of this it is very hard to reason about design proposals.
> > From the proposals so far I think we have a lot of discussion around
> > possible apis, but less around what the user needs for different use
> cases
> > and how they would implement that using the api.
> >
> > Here is an example:
> > You aggregate click and impression data for a reddit like site. Every ten
> > minutes you want to output a ranked list of the top 10 articles ranked by
> > clicks/impressions for each geographical area. I want to be able run this
> > in steady state as well as rerun to regenerate results (or catch up if it
> > crashes).
> >
> > There are a couple of tricky things that seem to make this hard with
> either
> > of the options proposed:
> > 1. If I emit this data using event time I have the problem described
> where
> > a geographical region with no new clicks or impressions will fail to
> output
> > results.
> > 2. If I emit this data using system time I have the problem that when
> > reprocessing data my window may not be ten minutes but 10 hours if my
> > processing is very fast so it dramatically changes the output.
> >
> > Maybe a hybrid solution works: I window by event time but trigger results
> > by system time for windows that have updated? Not really sure the details
> > of making that work. Does that work? Are there concrete examples where
> you
> > actually want the current behavior?
> >
> > -Jay
> >
> >
> > On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com>
> > wrote:
> >
> > > Hi All,
> > >
> > > Thanks for the KIP. We were also in need of a mechanism to trigger
> > > punctuate in the absence of events.
> > >
> > > As I described in [
> > > https://issues.apache.org/jira/browse/KAFKA-3514?
> > > focusedCommentId=15926036&page=com.atlassian.jira.
> > > plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
> > > ],
> > >
> > >    - Our approached involved using the event time by default.
> > >    - The method to check if there is any punctuate ready in the
> > >    PunctuationQueue is triggered via the any event received by the
> stream
> > >    tread, or at the polling intervals in the absence of any events.
> > >    - When we create Punctuate objects (which contains the next event
> time
> > >    for punctuation and interval), we also record the creation time
> > (system
> > >    time).
> > >    - While checking for maturity of Punctuate Schedule by
> mayBePunctuate
> > >    method, we also check if the system clock has elapsed the punctuate
> > >    interval since the schedule creation time.
> > >    - In the absence of any event, or in the absence of any event for
> one
> > >    topic in the partition group assigned to the stream task, the system
> > > time
> > >    will elapse the interval and we trigger a punctuate using the
> expected
> > >    punctuation event time.
> > >    - we then create the next punctuation schedule as punctuation event
> > time
> > >    + punctuation interval, [again recording the system time of creation
> > of
> > > the
> > >    schedule].
> > >
> > > We call this a Hybrid Punctuate. Of course, this approach has pros and
> > > cons.
> > > Pros
> > >
> > >    - Punctuates will happen in <punctuate interval> time duration at
> max
> > in
> > >    terms of system time.
> > >    - The semantics as a whole continues to revolve around event time.
> > >    - We can use the old data [old timestamps] to rerun any experiments
> or
> > >    tests.
> > >
> > > Cons
> > >
> > >    - In case the  <punctuate interval> is not a time duration [say
> > logical
> > >    time/event count], then the approach might not be meaningful.
> > >    - In case there is a case where we have to wait for an actual event
> > from
> > >    a low event rate partition in the partition group, this approach
> will
> > > jump
> > >    the gun.
> > >    - in case the event processing cannot catch up with the event rate
> and
> > >    the expected timestamp events gets queued for long time, this
> approach
> > >    might jump the gun.
> > >
> > > I believe the above approach and discussion goes close to the approach
> A.
> > >
> > > -----------
> > >
> > > I like the idea of having an even count based punctuate.
> > >
> > > -----------
> > >
> > > I agree with the discussion around approach C, that we should provide
> the
> > > user with the option to choose system time or event time based
> > punctuates.
> > > But I believe that the user predominantly wants to use event time while
> > not
> > > missing out on regular punctuates due to event delays or event
> absences.
> > > Hence a complex punctuate option as Matthias mentioned (quoted below)
> > would
> > > be most apt.
> > >
> > > "- We might want to add "complex" schedules later on (like, punctuate
> on
> > > every 10 seconds event-time or 60 seconds system-time whatever comes
> > > first)."
> > >
> > > -----------
> > >
> > > I think I read somewhere that Kafka Streams started with System Time as
> > the
> > > punctuation standard, but was later changed to Event Time. I guess
> there
> > > would be some good reason behind it. As Kafka Streams want to evolve
> more
> > > on the Stream Processing front, I believe the emphasis on event time
> > would
> > > remain quite strong.
> > >
> > >
> > > With Regards,
> > >
> > > Arun Mathew
> > > Yahoo! JAPAN Corporation, Tokyo
> > >
> > >
> > > On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com>
> wrote:
> > >
> > > > Yeah I like PuncutationType much better; I just threw Time out there
> > > > more as a strawman than an actual suggestion ;) I still think it's
> > > > worth considering what this buys us over an additional callback. I
> > > > foresee a number of punctuate implementations following this pattern:
> > > >
> > > > public void punctuate(PunctuationType type) {
> > > >     switch (type) {
> > > >         case EVENT_TIME:
> > > >             methodA();
> > > >             break;
> > > >         case SYSTEM_TIME:
> > > >             methodB();
> > > >             break;
> > > >     }
> > > > }
> > > >
> > > > I guess one advantage of this approach is we could add additional
> > > > punctuation types later in a backwards compatible way (like event
> count
> > > > as you mentioned).
> > > >
> > > > -Tommy
> > > >
> > > >
> > > > On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
> > > > > That sounds promising.
> > > > >
> > > > > I am just wondering if `Time` is the best name. Maybe we want to
> add
> > > > > other non-time based punctuations at some point later. I would
> > > > > suggest
> > > > >
> > > > > enum PunctuationType {
> > > > >   EVENT_TIME,
> > > > >   SYSTEM_TIME,
> > > > > }
> > > > >
> > > > > or similar. Just to keep the door open -- it's easier to add new
> > > > > stuff
> > > > > if the name is more generic.
> > > > >
> > > > >
> > > > > -Matthias
> > > > >
> > > > >
> > > > > On 4/4/17 5:30 AM, Thomas Becker wrote:
> > > > > >
> > > > > > I agree that the framework providing and managing the notion of
> > > > > > stream
> > > > > > time is valuable and not something we would want to delegate to
> the
> > > > > > tasks. I'm not entirely convinced that a separate callback
> (option
> > > > > > C)
> > > > > > is that messy (it could just be a default method with an empty
> > > > > > implementation), but if we wanted a single API to handle both
> > > > > > cases,
> > > > > > how about something like the following?
> > > > > >
> > > > > > enum Time {
> > > > > >    STREAM,
> > > > > >    CLOCK
> > > > > > }
> > > > > >
> > > > > > Then on ProcessorContext:
> > > > > > context.schedule(Time time, long interval)  // We could allow
> this
> > > > > > to
> > > > > > be called once for each value of time to mix approaches.
> > > > > >
> > > > > > Then the Processor API becomes:
> > > > > > punctuate(Time time) // time here denotes which schedule resulted
> > > > > > in
> > > > > > this call.
> > > > > >
> > > > > > Thoughts?
> > > > > >
> > > > > >
> > > > > > On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
> > > > > > >
> > > > > > > Thanks a lot for the KIP Michal,
> > > > > > >
> > > > > > > I was thinking about the four options you proposed in more
> > > > > > > details
> > > > > > > and
> > > > > > > this are my thoughts:
> > > > > > >
> > > > > > > (A) You argue, that users can still "punctuate" on event-time
> via
> > > > > > > process(), but I am not sure if this is possible. Note, that
> > > > > > > users
> > > > > > > only
> > > > > > > get record timestamps via context.timestamp(). Thus, users
> would
> > > > > > > need
> > > > > > > to
> > > > > > > track the time progress per partition (based on the partitions
> > > > > > > they
> > > > > > > obverse via context.partition(). (This alone puts a huge burden
> > > > > > > on
> > > > > > > the
> > > > > > > user by itself.) However, users are not notified at startup
> what
> > > > > > > partitions are assigned, and user are not notified when
> > > > > > > partitions
> > > > > > > get
> > > > > > > revoked. Because this information is not available, it's not
> > > > > > > possible
> > > > > > > to
> > > > > > > "manually advance" stream-time, and thus event-time punctuation
> > > > > > > within
> > > > > > > process() seems not to be possible -- or do you see a way to
> get
> > > > > > > it
> > > > > > > done? And even if, it might still be too clumsy to use.
> > > > > > >
> > > > > > > (B) This does not allow to mix both approaches, thus limiting
> > > > > > > what
> > > > > > > users
> > > > > > > can do.
> > > > > > >
> > > > > > > (C) This should give all flexibility we need. However, just
> > > > > > > adding
> > > > > > > one
> > > > > > > more method seems to be a solution that is too simple (cf my
> > > > > > > comments
> > > > > > > below).
> > > > > > >
> > > > > > > (D) This might be hard to use. Also, I am not sure how a user
> > > > > > > could
> > > > > > > enable system-time and event-time punctuation in parallel.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Overall options (C) seems to be the most promising approach to
> > > > > > > me.
> > > > > > > Because I also favor a clean API, we might keep current
> > > > > > > punctuate()
> > > > > > > as-is, but deprecate it -- so we can remove it at some later
> > > > > > > point
> > > > > > > when
> > > > > > > people use the "new punctuate API".
> > > > > > >
> > > > > > >
> > > > > > > Couple of follow up questions:
> > > > > > >
> > > > > > > - I am wondering, if we should have two callback methods or
> just
> > > > > > > one
> > > > > > > (ie, a unified for system and event time punctuation or one for
> > > > > > > each?).
> > > > > > >
> > > > > > > - If we have one, how can the user figure out, which condition
> > > > > > > did
> > > > > > > trigger?
> > > > > > >
> > > > > > > - How would the API look like, for registering different
> > > > > > > punctuate
> > > > > > > schedules? The "type" must be somehow defined?
> > > > > > >
> > > > > > > - We might want to add "complex" schedules later on (like,
> > > > > > > punctuate
> > > > > > > on
> > > > > > > every 10 seconds event-time or 60 seconds system-time whatever
> > > > > > > comes
> > > > > > > first). I don't say we should add this right away, but we might
> > > > > > > want
> > > > > > > to
> > > > > > > define the API in a way, that it allows extensions like this
> > > > > > > later
> > > > > > > on,
> > > > > > > without redesigning the API (ie, the API should be designed
> > > > > > > extensible)
> > > > > > >
> > > > > > > - Did you ever consider count-based punctuation?
> > > > > > >
> > > > > > >
> > > > > > > I understand, that you would like to solve a simple problem,
> but
> > > > > > > we
> > > > > > > learned from the past, that just "adding some API" quickly
> leads
> > > > > > > to a
> > > > > > > not very well defined API that needs time consuming clean up
> > > > > > > later on
> > > > > > > via other KIPs. Thus, I would prefer to get a holistic
> > > > > > > punctuation
> > > > > > > KIP
> > > > > > > with this from the beginning on to avoid later painful
> redesign.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > -Matthias
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> > > > > > > >
> > > > > > > >
> > > > > > > > Thanks Thomas,
> > > > > > > >
> > > > > > > > I'm also wary of changing the existing semantics of
> punctuate,
> > > > > > > > for
> > > > > > > > backward compatibility reasons, although I like the
> conceptual
> > > > > > > > simplicity of that option.
> > > > > > > >
> > > > > > > > Adding a new method to me feels safer but, in a way, uglier.
> I
> > > > > > > > added
> > > > > > > > this to the KIP now as option (C).
> > > > > > > >
> > > > > > > > The TimestampExtractor mechanism is actually more flexible,
> as
> > > > > > > > it
> > > > > > > > allows
> > > > > > > > you to return any value, you're not limited to event time or
> > > > > > > > system
> > > > > > > > time
> > > > > > > > (although I don't see an actual use case where you might need
> > > > > > > > anything
> > > > > > > > else then those two). Hence I also proposed the option to
> allow
> > > > > > > > users
> > > > > > > > to, effectively, decide what "stream time" is for them given
> > > > > > > > the
> > > > > > > > presence or absence of messages, much like they can decide
> what
> > > > > > > > msg
> > > > > > > > time
> > > > > > > > means for them using the TimestampExtractor. What do you
> think
> > > > > > > > about
> > > > > > > > that? This is probably most flexible but also most
> complicated.
> > > > > > > >
> > > > > > > > All comments appreciated.
> > > > > > > >
> > > > > > > > Cheers,
> > > > > > > >
> > > > > > > > Michal
> > > > > > > >
> > > > > > > >
> > > > > > > > On 03/04/17 19:23, Thomas Becker wrote:
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Although I fully agree we need a way to trigger periodic
> > > > > > > > > processing
> > > > > > > > > that is independent from whether and when messages arrive,
> > > > > > > > > I'm
> > > > > > > > > not sure
> > > > > > > > > I like the idea of changing the existing semantics across
> the
> > > > > > > > > board.
> > > > > > > > > What if we added an additional callback to Processor that
> can
> > > > > > > > > be
> > > > > > > > > scheduled similarly to punctuate() but was always called at
> > > > > > > > > fixed, wall
> > > > > > > > > clock based intervals? This way you wouldn't have to give
> up
> > > > > > > > > the
> > > > > > > > > notion
> > > > > > > > > of stream time to be able to do periodic processing.
> > > > > > > > >
> > > > > > > > > On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Hi all,
> > > > > > > > > >
> > > > > > > > > > I have created a draft for KIP-138: Change punctuate
> > > > > > > > > > semantics
> > > > > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 138%
> > > > > > > > > > 3A+C
> > > > > > > > > > hange+
> > > > > > > > > > punctuate+semantics>
> > > > > > > > > > .
> > > > > > > > > >
> > > > > > > > > > Appreciating there can be different views on system-time
> vs
> > > > > > > > > > event-
> > > > > > > > > > time
> > > > > > > > > > semantics for punctuation depending on use-case and the
> > > > > > > > > > importance of
> > > > > > > > > > backwards compatibility of any such change, I've left it
> > > > > > > > > > quite
> > > > > > > > > > open
> > > > > > > > > > and
> > > > > > > > > > hope to fill in more info as the discussion progresses.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Michal
> > > > > > --
> > > > > >
> > > > > >
> > > > > >     Tommy Becker
> > > > > >
> > > > > >     Senior Software Engineer
> > > > > >
> > > > > >     O +1 919.460.4747
> > > > > >
> > > > > >     tivo.com
> > > > > >
> > > > > >
> > > > > > ________________________________
> > > > > >
> > > > > > This email and any attachments may contain confidential and
> > > > > > privileged material for the sole use of the intended recipient.
> Any
> > > > > > review, copying, or distribution of this email (or any
> attachments)
> > > > > > by others is prohibited. If you are not the intended recipient,
> > > > > > please contact the sender immediately and permanently delete this
> > > > > > email and any attachments. No employee or agent of TiVo Inc. is
> > > > > > authorized to conclude any binding agreement on behalf of TiVo
> Inc.
> > > > > > by email. Binding agreements with TiVo Inc. may only be made by a
> > > > > > signed written agreement.
> > > > > >
> > > > --
> > > >
> > > >
> > > >     Tommy Becker
> > > >
> > > >     Senior Software Engineer
> > > >
> > > >     O +1 919.460.4747
> > > >
> > > >     tivo.com
> > > >
> > > >
> > > > ________________________________
> > > >
> > > > This email and any attachments may contain confidential and
> privileged
> > > > material for the sole use of the intended recipient. Any review,
> > copying,
> > > > or distribution of this email (or any attachments) by others is
> > > prohibited.
> > > > If you are not the intended recipient, please contact the sender
> > > > immediately and permanently delete this email and any attachments. No
> > > > employee or agent of TiVo Inc. is authorized to conclude any binding
> > > > agreement on behalf of TiVo Inc. by email. Binding agreements with
> TiVo
> > > > Inc. may only be made by a signed written agreement.
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Tianji Li <sk...@gmail.com>.
Hi Jay,

The hybrid solution is exactly what I expect and need for our use cases
when dealing with telecom data.

Thanks
Tianji

On Wed, Apr 5, 2017 at 12:01 AM, Jay Kreps <ja...@confluent.io> wrote:

> Hey guys,
>
> One thing I've always found super important for this kind of design work is
> to do a really good job of cataloging the landscape of use cases and how
> prevalent each one is. By that I mean not just listing lots of uses, but
> also grouping them into categories that functionally need the same thing.
> In the absence of this it is very hard to reason about design proposals.
> From the proposals so far I think we have a lot of discussion around
> possible apis, but less around what the user needs for different use cases
> and how they would implement that using the api.
>
> Here is an example:
> You aggregate click and impression data for a reddit like site. Every ten
> minutes you want to output a ranked list of the top 10 articles ranked by
> clicks/impressions for each geographical area. I want to be able run this
> in steady state as well as rerun to regenerate results (or catch up if it
> crashes).
>
> There are a couple of tricky things that seem to make this hard with either
> of the options proposed:
> 1. If I emit this data using event time I have the problem described where
> a geographical region with no new clicks or impressions will fail to output
> results.
> 2. If I emit this data using system time I have the problem that when
> reprocessing data my window may not be ten minutes but 10 hours if my
> processing is very fast so it dramatically changes the output.
>
> Maybe a hybrid solution works: I window by event time but trigger results
> by system time for windows that have updated? Not really sure the details
> of making that work. Does that work? Are there concrete examples where you
> actually want the current behavior?
>
> -Jay
>
>
> On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com>
> wrote:
>
> > Hi All,
> >
> > Thanks for the KIP. We were also in need of a mechanism to trigger
> > punctuate in the absence of events.
> >
> > As I described in [
> > https://issues.apache.org/jira/browse/KAFKA-3514?
> > focusedCommentId=15926036&page=com.atlassian.jira.
> > plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
> > ],
> >
> >    - Our approached involved using the event time by default.
> >    - The method to check if there is any punctuate ready in the
> >    PunctuationQueue is triggered via the any event received by the stream
> >    tread, or at the polling intervals in the absence of any events.
> >    - When we create Punctuate objects (which contains the next event time
> >    for punctuation and interval), we also record the creation time
> (system
> >    time).
> >    - While checking for maturity of Punctuate Schedule by mayBePunctuate
> >    method, we also check if the system clock has elapsed the punctuate
> >    interval since the schedule creation time.
> >    - In the absence of any event, or in the absence of any event for one
> >    topic in the partition group assigned to the stream task, the system
> > time
> >    will elapse the interval and we trigger a punctuate using the expected
> >    punctuation event time.
> >    - we then create the next punctuation schedule as punctuation event
> time
> >    + punctuation interval, [again recording the system time of creation
> of
> > the
> >    schedule].
> >
> > We call this a Hybrid Punctuate. Of course, this approach has pros and
> > cons.
> > Pros
> >
> >    - Punctuates will happen in <punctuate interval> time duration at max
> in
> >    terms of system time.
> >    - The semantics as a whole continues to revolve around event time.
> >    - We can use the old data [old timestamps] to rerun any experiments or
> >    tests.
> >
> > Cons
> >
> >    - In case the  <punctuate interval> is not a time duration [say
> logical
> >    time/event count], then the approach might not be meaningful.
> >    - In case there is a case where we have to wait for an actual event
> from
> >    a low event rate partition in the partition group, this approach will
> > jump
> >    the gun.
> >    - in case the event processing cannot catch up with the event rate and
> >    the expected timestamp events gets queued for long time, this approach
> >    might jump the gun.
> >
> > I believe the above approach and discussion goes close to the approach A.
> >
> > -----------
> >
> > I like the idea of having an even count based punctuate.
> >
> > -----------
> >
> > I agree with the discussion around approach C, that we should provide the
> > user with the option to choose system time or event time based
> punctuates.
> > But I believe that the user predominantly wants to use event time while
> not
> > missing out on regular punctuates due to event delays or event absences.
> > Hence a complex punctuate option as Matthias mentioned (quoted below)
> would
> > be most apt.
> >
> > "- We might want to add "complex" schedules later on (like, punctuate on
> > every 10 seconds event-time or 60 seconds system-time whatever comes
> > first)."
> >
> > -----------
> >
> > I think I read somewhere that Kafka Streams started with System Time as
> the
> > punctuation standard, but was later changed to Event Time. I guess there
> > would be some good reason behind it. As Kafka Streams want to evolve more
> > on the Stream Processing front, I believe the emphasis on event time
> would
> > remain quite strong.
> >
> >
> > With Regards,
> >
> > Arun Mathew
> > Yahoo! JAPAN Corporation, Tokyo
> >
> >
> > On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com> wrote:
> >
> > > Yeah I like PuncutationType much better; I just threw Time out there
> > > more as a strawman than an actual suggestion ;) I still think it's
> > > worth considering what this buys us over an additional callback. I
> > > foresee a number of punctuate implementations following this pattern:
> > >
> > > public void punctuate(PunctuationType type) {
> > >     switch (type) {
> > >         case EVENT_TIME:
> > >             methodA();
> > >             break;
> > >         case SYSTEM_TIME:
> > >             methodB();
> > >             break;
> > >     }
> > > }
> > >
> > > I guess one advantage of this approach is we could add additional
> > > punctuation types later in a backwards compatible way (like event count
> > > as you mentioned).
> > >
> > > -Tommy
> > >
> > >
> > > On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
> > > > That sounds promising.
> > > >
> > > > I am just wondering if `Time` is the best name. Maybe we want to add
> > > > other non-time based punctuations at some point later. I would
> > > > suggest
> > > >
> > > > enum PunctuationType {
> > > >   EVENT_TIME,
> > > >   SYSTEM_TIME,
> > > > }
> > > >
> > > > or similar. Just to keep the door open -- it's easier to add new
> > > > stuff
> > > > if the name is more generic.
> > > >
> > > >
> > > > -Matthias
> > > >
> > > >
> > > > On 4/4/17 5:30 AM, Thomas Becker wrote:
> > > > >
> > > > > I agree that the framework providing and managing the notion of
> > > > > stream
> > > > > time is valuable and not something we would want to delegate to the
> > > > > tasks. I'm not entirely convinced that a separate callback (option
> > > > > C)
> > > > > is that messy (it could just be a default method with an empty
> > > > > implementation), but if we wanted a single API to handle both
> > > > > cases,
> > > > > how about something like the following?
> > > > >
> > > > > enum Time {
> > > > >    STREAM,
> > > > >    CLOCK
> > > > > }
> > > > >
> > > > > Then on ProcessorContext:
> > > > > context.schedule(Time time, long interval)  // We could allow this
> > > > > to
> > > > > be called once for each value of time to mix approaches.
> > > > >
> > > > > Then the Processor API becomes:
> > > > > punctuate(Time time) // time here denotes which schedule resulted
> > > > > in
> > > > > this call.
> > > > >
> > > > > Thoughts?
> > > > >
> > > > >
> > > > > On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
> > > > > >
> > > > > > Thanks a lot for the KIP Michal,
> > > > > >
> > > > > > I was thinking about the four options you proposed in more
> > > > > > details
> > > > > > and
> > > > > > this are my thoughts:
> > > > > >
> > > > > > (A) You argue, that users can still "punctuate" on event-time via
> > > > > > process(), but I am not sure if this is possible. Note, that
> > > > > > users
> > > > > > only
> > > > > > get record timestamps via context.timestamp(). Thus, users would
> > > > > > need
> > > > > > to
> > > > > > track the time progress per partition (based on the partitions
> > > > > > they
> > > > > > obverse via context.partition(). (This alone puts a huge burden
> > > > > > on
> > > > > > the
> > > > > > user by itself.) However, users are not notified at startup what
> > > > > > partitions are assigned, and user are not notified when
> > > > > > partitions
> > > > > > get
> > > > > > revoked. Because this information is not available, it's not
> > > > > > possible
> > > > > > to
> > > > > > "manually advance" stream-time, and thus event-time punctuation
> > > > > > within
> > > > > > process() seems not to be possible -- or do you see a way to get
> > > > > > it
> > > > > > done? And even if, it might still be too clumsy to use.
> > > > > >
> > > > > > (B) This does not allow to mix both approaches, thus limiting
> > > > > > what
> > > > > > users
> > > > > > can do.
> > > > > >
> > > > > > (C) This should give all flexibility we need. However, just
> > > > > > adding
> > > > > > one
> > > > > > more method seems to be a solution that is too simple (cf my
> > > > > > comments
> > > > > > below).
> > > > > >
> > > > > > (D) This might be hard to use. Also, I am not sure how a user
> > > > > > could
> > > > > > enable system-time and event-time punctuation in parallel.
> > > > > >
> > > > > >
> > > > > >
> > > > > > Overall options (C) seems to be the most promising approach to
> > > > > > me.
> > > > > > Because I also favor a clean API, we might keep current
> > > > > > punctuate()
> > > > > > as-is, but deprecate it -- so we can remove it at some later
> > > > > > point
> > > > > > when
> > > > > > people use the "new punctuate API".
> > > > > >
> > > > > >
> > > > > > Couple of follow up questions:
> > > > > >
> > > > > > - I am wondering, if we should have two callback methods or just
> > > > > > one
> > > > > > (ie, a unified for system and event time punctuation or one for
> > > > > > each?).
> > > > > >
> > > > > > - If we have one, how can the user figure out, which condition
> > > > > > did
> > > > > > trigger?
> > > > > >
> > > > > > - How would the API look like, for registering different
> > > > > > punctuate
> > > > > > schedules? The "type" must be somehow defined?
> > > > > >
> > > > > > - We might want to add "complex" schedules later on (like,
> > > > > > punctuate
> > > > > > on
> > > > > > every 10 seconds event-time or 60 seconds system-time whatever
> > > > > > comes
> > > > > > first). I don't say we should add this right away, but we might
> > > > > > want
> > > > > > to
> > > > > > define the API in a way, that it allows extensions like this
> > > > > > later
> > > > > > on,
> > > > > > without redesigning the API (ie, the API should be designed
> > > > > > extensible)
> > > > > >
> > > > > > - Did you ever consider count-based punctuation?
> > > > > >
> > > > > >
> > > > > > I understand, that you would like to solve a simple problem, but
> > > > > > we
> > > > > > learned from the past, that just "adding some API" quickly leads
> > > > > > to a
> > > > > > not very well defined API that needs time consuming clean up
> > > > > > later on
> > > > > > via other KIPs. Thus, I would prefer to get a holistic
> > > > > > punctuation
> > > > > > KIP
> > > > > > with this from the beginning on to avoid later painful redesign.
> > > > > >
> > > > > >
> > > > > >
> > > > > > -Matthias
> > > > > >
> > > > > >
> > > > > >
> > > > > > On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> > > > > > >
> > > > > > >
> > > > > > > Thanks Thomas,
> > > > > > >
> > > > > > > I'm also wary of changing the existing semantics of punctuate,
> > > > > > > for
> > > > > > > backward compatibility reasons, although I like the conceptual
> > > > > > > simplicity of that option.
> > > > > > >
> > > > > > > Adding a new method to me feels safer but, in a way, uglier. I
> > > > > > > added
> > > > > > > this to the KIP now as option (C).
> > > > > > >
> > > > > > > The TimestampExtractor mechanism is actually more flexible, as
> > > > > > > it
> > > > > > > allows
> > > > > > > you to return any value, you're not limited to event time or
> > > > > > > system
> > > > > > > time
> > > > > > > (although I don't see an actual use case where you might need
> > > > > > > anything
> > > > > > > else then those two). Hence I also proposed the option to allow
> > > > > > > users
> > > > > > > to, effectively, decide what "stream time" is for them given
> > > > > > > the
> > > > > > > presence or absence of messages, much like they can decide what
> > > > > > > msg
> > > > > > > time
> > > > > > > means for them using the TimestampExtractor. What do you think
> > > > > > > about
> > > > > > > that? This is probably most flexible but also most complicated.
> > > > > > >
> > > > > > > All comments appreciated.
> > > > > > >
> > > > > > > Cheers,
> > > > > > >
> > > > > > > Michal
> > > > > > >
> > > > > > >
> > > > > > > On 03/04/17 19:23, Thomas Becker wrote:
> > > > > > > >
> > > > > > > >
> > > > > > > > Although I fully agree we need a way to trigger periodic
> > > > > > > > processing
> > > > > > > > that is independent from whether and when messages arrive,
> > > > > > > > I'm
> > > > > > > > not sure
> > > > > > > > I like the idea of changing the existing semantics across the
> > > > > > > > board.
> > > > > > > > What if we added an additional callback to Processor that can
> > > > > > > > be
> > > > > > > > scheduled similarly to punctuate() but was always called at
> > > > > > > > fixed, wall
> > > > > > > > clock based intervals? This way you wouldn't have to give up
> > > > > > > > the
> > > > > > > > notion
> > > > > > > > of stream time to be able to do periodic processing.
> > > > > > > >
> > > > > > > > On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Hi all,
> > > > > > > > >
> > > > > > > > > I have created a draft for KIP-138: Change punctuate
> > > > > > > > > semantics
> > > > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 138%
> > > > > > > > > 3A+C
> > > > > > > > > hange+
> > > > > > > > > punctuate+semantics>
> > > > > > > > > .
> > > > > > > > >
> > > > > > > > > Appreciating there can be different views on system-time vs
> > > > > > > > > event-
> > > > > > > > > time
> > > > > > > > > semantics for punctuation depending on use-case and the
> > > > > > > > > importance of
> > > > > > > > > backwards compatibility of any such change, I've left it
> > > > > > > > > quite
> > > > > > > > > open
> > > > > > > > > and
> > > > > > > > > hope to fill in more info as the discussion progresses.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Michal
> > > > > --
> > > > >
> > > > >
> > > > >     Tommy Becker
> > > > >
> > > > >     Senior Software Engineer
> > > > >
> > > > >     O +1 919.460.4747
> > > > >
> > > > >     tivo.com
> > > > >
> > > > >
> > > > > ________________________________
> > > > >
> > > > > This email and any attachments may contain confidential and
> > > > > privileged material for the sole use of the intended recipient. Any
> > > > > review, copying, or distribution of this email (or any attachments)
> > > > > by others is prohibited. If you are not the intended recipient,
> > > > > please contact the sender immediately and permanently delete this
> > > > > email and any attachments. No employee or agent of TiVo Inc. is
> > > > > authorized to conclude any binding agreement on behalf of TiVo Inc.
> > > > > by email. Binding agreements with TiVo Inc. may only be made by a
> > > > > signed written agreement.
> > > > >
> > > --
> > >
> > >
> > >     Tommy Becker
> > >
> > >     Senior Software Engineer
> > >
> > >     O +1 919.460.4747
> > >
> > >     tivo.com
> > >
> > >
> > > ________________________________
> > >
> > > This email and any attachments may contain confidential and privileged
> > > material for the sole use of the intended recipient. Any review,
> copying,
> > > or distribution of this email (or any attachments) by others is
> > prohibited.
> > > If you are not the intended recipient, please contact the sender
> > > immediately and permanently delete this email and any attachments. No
> > > employee or agent of TiVo Inc. is authorized to conclude any binding
> > > agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> > > Inc. may only be made by a signed written agreement.
> > >
> >
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Jay Kreps <ja...@confluent.io>.
Hey guys,

One thing I've always found super important for this kind of design work is
to do a really good job of cataloging the landscape of use cases and how
prevalent each one is. By that I mean not just listing lots of uses, but
also grouping them into categories that functionally need the same thing.
In the absence of this it is very hard to reason about design proposals.
From the proposals so far I think we have a lot of discussion around
possible apis, but less around what the user needs for different use cases
and how they would implement that using the api.

Here is an example:
You aggregate click and impression data for a reddit like site. Every ten
minutes you want to output a ranked list of the top 10 articles ranked by
clicks/impressions for each geographical area. I want to be able run this
in steady state as well as rerun to regenerate results (or catch up if it
crashes).

There are a couple of tricky things that seem to make this hard with either
of the options proposed:
1. If I emit this data using event time I have the problem described where
a geographical region with no new clicks or impressions will fail to output
results.
2. If I emit this data using system time I have the problem that when
reprocessing data my window may not be ten minutes but 10 hours if my
processing is very fast so it dramatically changes the output.

Maybe a hybrid solution works: I window by event time but trigger results
by system time for windows that have updated? Not really sure the details
of making that work. Does that work? Are there concrete examples where you
actually want the current behavior?

-Jay


On Tue, Apr 4, 2017 at 8:32 PM, Arun Mathew <ar...@gmail.com> wrote:

> Hi All,
>
> Thanks for the KIP. We were also in need of a mechanism to trigger
> punctuate in the absence of events.
>
> As I described in [
> https://issues.apache.org/jira/browse/KAFKA-3514?
> focusedCommentId=15926036&page=com.atlassian.jira.
> plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
> ],
>
>    - Our approached involved using the event time by default.
>    - The method to check if there is any punctuate ready in the
>    PunctuationQueue is triggered via the any event received by the stream
>    tread, or at the polling intervals in the absence of any events.
>    - When we create Punctuate objects (which contains the next event time
>    for punctuation and interval), we also record the creation time (system
>    time).
>    - While checking for maturity of Punctuate Schedule by mayBePunctuate
>    method, we also check if the system clock has elapsed the punctuate
>    interval since the schedule creation time.
>    - In the absence of any event, or in the absence of any event for one
>    topic in the partition group assigned to the stream task, the system
> time
>    will elapse the interval and we trigger a punctuate using the expected
>    punctuation event time.
>    - we then create the next punctuation schedule as punctuation event time
>    + punctuation interval, [again recording the system time of creation of
> the
>    schedule].
>
> We call this a Hybrid Punctuate. Of course, this approach has pros and
> cons.
> Pros
>
>    - Punctuates will happen in <punctuate interval> time duration at max in
>    terms of system time.
>    - The semantics as a whole continues to revolve around event time.
>    - We can use the old data [old timestamps] to rerun any experiments or
>    tests.
>
> Cons
>
>    - In case the  <punctuate interval> is not a time duration [say logical
>    time/event count], then the approach might not be meaningful.
>    - In case there is a case where we have to wait for an actual event from
>    a low event rate partition in the partition group, this approach will
> jump
>    the gun.
>    - in case the event processing cannot catch up with the event rate and
>    the expected timestamp events gets queued for long time, this approach
>    might jump the gun.
>
> I believe the above approach and discussion goes close to the approach A.
>
> -----------
>
> I like the idea of having an even count based punctuate.
>
> -----------
>
> I agree with the discussion around approach C, that we should provide the
> user with the option to choose system time or event time based punctuates.
> But I believe that the user predominantly wants to use event time while not
> missing out on regular punctuates due to event delays or event absences.
> Hence a complex punctuate option as Matthias mentioned (quoted below) would
> be most apt.
>
> "- We might want to add "complex" schedules later on (like, punctuate on
> every 10 seconds event-time or 60 seconds system-time whatever comes
> first)."
>
> -----------
>
> I think I read somewhere that Kafka Streams started with System Time as the
> punctuation standard, but was later changed to Event Time. I guess there
> would be some good reason behind it. As Kafka Streams want to evolve more
> on the Stream Processing front, I believe the emphasis on event time would
> remain quite strong.
>
>
> With Regards,
>
> Arun Mathew
> Yahoo! JAPAN Corporation, Tokyo
>
>
> On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com> wrote:
>
> > Yeah I like PuncutationType much better; I just threw Time out there
> > more as a strawman than an actual suggestion ;) I still think it's
> > worth considering what this buys us over an additional callback. I
> > foresee a number of punctuate implementations following this pattern:
> >
> > public void punctuate(PunctuationType type) {
> >     switch (type) {
> >         case EVENT_TIME:
> >             methodA();
> >             break;
> >         case SYSTEM_TIME:
> >             methodB();
> >             break;
> >     }
> > }
> >
> > I guess one advantage of this approach is we could add additional
> > punctuation types later in a backwards compatible way (like event count
> > as you mentioned).
> >
> > -Tommy
> >
> >
> > On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
> > > That sounds promising.
> > >
> > > I am just wondering if `Time` is the best name. Maybe we want to add
> > > other non-time based punctuations at some point later. I would
> > > suggest
> > >
> > > enum PunctuationType {
> > >   EVENT_TIME,
> > >   SYSTEM_TIME,
> > > }
> > >
> > > or similar. Just to keep the door open -- it's easier to add new
> > > stuff
> > > if the name is more generic.
> > >
> > >
> > > -Matthias
> > >
> > >
> > > On 4/4/17 5:30 AM, Thomas Becker wrote:
> > > >
> > > > I agree that the framework providing and managing the notion of
> > > > stream
> > > > time is valuable and not something we would want to delegate to the
> > > > tasks. I'm not entirely convinced that a separate callback (option
> > > > C)
> > > > is that messy (it could just be a default method with an empty
> > > > implementation), but if we wanted a single API to handle both
> > > > cases,
> > > > how about something like the following?
> > > >
> > > > enum Time {
> > > >    STREAM,
> > > >    CLOCK
> > > > }
> > > >
> > > > Then on ProcessorContext:
> > > > context.schedule(Time time, long interval)  // We could allow this
> > > > to
> > > > be called once for each value of time to mix approaches.
> > > >
> > > > Then the Processor API becomes:
> > > > punctuate(Time time) // time here denotes which schedule resulted
> > > > in
> > > > this call.
> > > >
> > > > Thoughts?
> > > >
> > > >
> > > > On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
> > > > >
> > > > > Thanks a lot for the KIP Michal,
> > > > >
> > > > > I was thinking about the four options you proposed in more
> > > > > details
> > > > > and
> > > > > this are my thoughts:
> > > > >
> > > > > (A) You argue, that users can still "punctuate" on event-time via
> > > > > process(), but I am not sure if this is possible. Note, that
> > > > > users
> > > > > only
> > > > > get record timestamps via context.timestamp(). Thus, users would
> > > > > need
> > > > > to
> > > > > track the time progress per partition (based on the partitions
> > > > > they
> > > > > obverse via context.partition(). (This alone puts a huge burden
> > > > > on
> > > > > the
> > > > > user by itself.) However, users are not notified at startup what
> > > > > partitions are assigned, and user are not notified when
> > > > > partitions
> > > > > get
> > > > > revoked. Because this information is not available, it's not
> > > > > possible
> > > > > to
> > > > > "manually advance" stream-time, and thus event-time punctuation
> > > > > within
> > > > > process() seems not to be possible -- or do you see a way to get
> > > > > it
> > > > > done? And even if, it might still be too clumsy to use.
> > > > >
> > > > > (B) This does not allow to mix both approaches, thus limiting
> > > > > what
> > > > > users
> > > > > can do.
> > > > >
> > > > > (C) This should give all flexibility we need. However, just
> > > > > adding
> > > > > one
> > > > > more method seems to be a solution that is too simple (cf my
> > > > > comments
> > > > > below).
> > > > >
> > > > > (D) This might be hard to use. Also, I am not sure how a user
> > > > > could
> > > > > enable system-time and event-time punctuation in parallel.
> > > > >
> > > > >
> > > > >
> > > > > Overall options (C) seems to be the most promising approach to
> > > > > me.
> > > > > Because I also favor a clean API, we might keep current
> > > > > punctuate()
> > > > > as-is, but deprecate it -- so we can remove it at some later
> > > > > point
> > > > > when
> > > > > people use the "new punctuate API".
> > > > >
> > > > >
> > > > > Couple of follow up questions:
> > > > >
> > > > > - I am wondering, if we should have two callback methods or just
> > > > > one
> > > > > (ie, a unified for system and event time punctuation or one for
> > > > > each?).
> > > > >
> > > > > - If we have one, how can the user figure out, which condition
> > > > > did
> > > > > trigger?
> > > > >
> > > > > - How would the API look like, for registering different
> > > > > punctuate
> > > > > schedules? The "type" must be somehow defined?
> > > > >
> > > > > - We might want to add "complex" schedules later on (like,
> > > > > punctuate
> > > > > on
> > > > > every 10 seconds event-time or 60 seconds system-time whatever
> > > > > comes
> > > > > first). I don't say we should add this right away, but we might
> > > > > want
> > > > > to
> > > > > define the API in a way, that it allows extensions like this
> > > > > later
> > > > > on,
> > > > > without redesigning the API (ie, the API should be designed
> > > > > extensible)
> > > > >
> > > > > - Did you ever consider count-based punctuation?
> > > > >
> > > > >
> > > > > I understand, that you would like to solve a simple problem, but
> > > > > we
> > > > > learned from the past, that just "adding some API" quickly leads
> > > > > to a
> > > > > not very well defined API that needs time consuming clean up
> > > > > later on
> > > > > via other KIPs. Thus, I would prefer to get a holistic
> > > > > punctuation
> > > > > KIP
> > > > > with this from the beginning on to avoid later painful redesign.
> > > > >
> > > > >
> > > > >
> > > > > -Matthias
> > > > >
> > > > >
> > > > >
> > > > > On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> > > > > >
> > > > > >
> > > > > > Thanks Thomas,
> > > > > >
> > > > > > I'm also wary of changing the existing semantics of punctuate,
> > > > > > for
> > > > > > backward compatibility reasons, although I like the conceptual
> > > > > > simplicity of that option.
> > > > > >
> > > > > > Adding a new method to me feels safer but, in a way, uglier. I
> > > > > > added
> > > > > > this to the KIP now as option (C).
> > > > > >
> > > > > > The TimestampExtractor mechanism is actually more flexible, as
> > > > > > it
> > > > > > allows
> > > > > > you to return any value, you're not limited to event time or
> > > > > > system
> > > > > > time
> > > > > > (although I don't see an actual use case where you might need
> > > > > > anything
> > > > > > else then those two). Hence I also proposed the option to allow
> > > > > > users
> > > > > > to, effectively, decide what "stream time" is for them given
> > > > > > the
> > > > > > presence or absence of messages, much like they can decide what
> > > > > > msg
> > > > > > time
> > > > > > means for them using the TimestampExtractor. What do you think
> > > > > > about
> > > > > > that? This is probably most flexible but also most complicated.
> > > > > >
> > > > > > All comments appreciated.
> > > > > >
> > > > > > Cheers,
> > > > > >
> > > > > > Michal
> > > > > >
> > > > > >
> > > > > > On 03/04/17 19:23, Thomas Becker wrote:
> > > > > > >
> > > > > > >
> > > > > > > Although I fully agree we need a way to trigger periodic
> > > > > > > processing
> > > > > > > that is independent from whether and when messages arrive,
> > > > > > > I'm
> > > > > > > not sure
> > > > > > > I like the idea of changing the existing semantics across the
> > > > > > > board.
> > > > > > > What if we added an additional callback to Processor that can
> > > > > > > be
> > > > > > > scheduled similarly to punctuate() but was always called at
> > > > > > > fixed, wall
> > > > > > > clock based intervals? This way you wouldn't have to give up
> > > > > > > the
> > > > > > > notion
> > > > > > > of stream time to be able to do periodic processing.
> > > > > > >
> > > > > > > On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
> > > > > > > >
> > > > > > > >
> > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > I have created a draft for KIP-138: Change punctuate
> > > > > > > > semantics
> > > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%
> > > > > > > > 3A+C
> > > > > > > > hange+
> > > > > > > > punctuate+semantics>
> > > > > > > > .
> > > > > > > >
> > > > > > > > Appreciating there can be different views on system-time vs
> > > > > > > > event-
> > > > > > > > time
> > > > > > > > semantics for punctuation depending on use-case and the
> > > > > > > > importance of
> > > > > > > > backwards compatibility of any such change, I've left it
> > > > > > > > quite
> > > > > > > > open
> > > > > > > > and
> > > > > > > > hope to fill in more info as the discussion progresses.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Michal
> > > > --
> > > >
> > > >
> > > >     Tommy Becker
> > > >
> > > >     Senior Software Engineer
> > > >
> > > >     O +1 919.460.4747
> > > >
> > > >     tivo.com
> > > >
> > > >
> > > > ________________________________
> > > >
> > > > This email and any attachments may contain confidential and
> > > > privileged material for the sole use of the intended recipient. Any
> > > > review, copying, or distribution of this email (or any attachments)
> > > > by others is prohibited. If you are not the intended recipient,
> > > > please contact the sender immediately and permanently delete this
> > > > email and any attachments. No employee or agent of TiVo Inc. is
> > > > authorized to conclude any binding agreement on behalf of TiVo Inc.
> > > > by email. Binding agreements with TiVo Inc. may only be made by a
> > > > signed written agreement.
> > > >
> > --
> >
> >
> >     Tommy Becker
> >
> >     Senior Software Engineer
> >
> >     O +1 919.460.4747
> >
> >     tivo.com
> >
> >
> > ________________________________
> >
> > This email and any attachments may contain confidential and privileged
> > material for the sole use of the intended recipient. Any review, copying,
> > or distribution of this email (or any attachments) by others is
> prohibited.
> > If you are not the intended recipient, please contact the sender
> > immediately and permanently delete this email and any attachments. No
> > employee or agent of TiVo Inc. is authorized to conclude any binding
> > agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> > Inc. may only be made by a signed written agreement.
> >
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Arun Mathew <ar...@gmail.com>.
Hi All,

Thanks for the KIP. We were also in need of a mechanism to trigger
punctuate in the absence of events.

As I described in [
https://issues.apache.org/jira/browse/KAFKA-3514?focusedCommentId=15926036&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15926036
],

   - Our approached involved using the event time by default.
   - The method to check if there is any punctuate ready in the
   PunctuationQueue is triggered via the any event received by the stream
   tread, or at the polling intervals in the absence of any events.
   - When we create Punctuate objects (which contains the next event time
   for punctuation and interval), we also record the creation time (system
   time).
   - While checking for maturity of Punctuate Schedule by mayBePunctuate
   method, we also check if the system clock has elapsed the punctuate
   interval since the schedule creation time.
   - In the absence of any event, or in the absence of any event for one
   topic in the partition group assigned to the stream task, the system time
   will elapse the interval and we trigger a punctuate using the expected
   punctuation event time.
   - we then create the next punctuation schedule as punctuation event time
   + punctuation interval, [again recording the system time of creation of the
   schedule].

We call this a Hybrid Punctuate. Of course, this approach has pros and cons.
Pros

   - Punctuates will happen in <punctuate interval> time duration at max in
   terms of system time.
   - The semantics as a whole continues to revolve around event time.
   - We can use the old data [old timestamps] to rerun any experiments or
   tests.

Cons

   - In case the  <punctuate interval> is not a time duration [say logical
   time/event count], then the approach might not be meaningful.
   - In case there is a case where we have to wait for an actual event from
   a low event rate partition in the partition group, this approach will jump
   the gun.
   - in case the event processing cannot catch up with the event rate and
   the expected timestamp events gets queued for long time, this approach
   might jump the gun.

I believe the above approach and discussion goes close to the approach A.

-----------

I like the idea of having an even count based punctuate.

-----------

I agree with the discussion around approach C, that we should provide the
user with the option to choose system time or event time based punctuates.
But I believe that the user predominantly wants to use event time while not
missing out on regular punctuates due to event delays or event absences.
Hence a complex punctuate option as Matthias mentioned (quoted below) would
be most apt.

"- We might want to add "complex" schedules later on (like, punctuate on
every 10 seconds event-time or 60 seconds system-time whatever comes
first)."

-----------

I think I read somewhere that Kafka Streams started with System Time as the
punctuation standard, but was later changed to Event Time. I guess there
would be some good reason behind it. As Kafka Streams want to evolve more
on the Stream Processing front, I believe the emphasis on event time would
remain quite strong.


With Regards,

Arun Mathew
Yahoo! JAPAN Corporation, Tokyo


On Wed, Apr 5, 2017 at 3:53 AM, Thomas Becker <to...@tivo.com> wrote:

> Yeah I like PuncutationType much better; I just threw Time out there
> more as a strawman than an actual suggestion ;) I still think it's
> worth considering what this buys us over an additional callback. I
> foresee a number of punctuate implementations following this pattern:
>
> public void punctuate(PunctuationType type) {
>     switch (type) {
>         case EVENT_TIME:
>             methodA();
>             break;
>         case SYSTEM_TIME:
>             methodB();
>             break;
>     }
> }
>
> I guess one advantage of this approach is we could add additional
> punctuation types later in a backwards compatible way (like event count
> as you mentioned).
>
> -Tommy
>
>
> On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
> > That sounds promising.
> >
> > I am just wondering if `Time` is the best name. Maybe we want to add
> > other non-time based punctuations at some point later. I would
> > suggest
> >
> > enum PunctuationType {
> >   EVENT_TIME,
> >   SYSTEM_TIME,
> > }
> >
> > or similar. Just to keep the door open -- it's easier to add new
> > stuff
> > if the name is more generic.
> >
> >
> > -Matthias
> >
> >
> > On 4/4/17 5:30 AM, Thomas Becker wrote:
> > >
> > > I agree that the framework providing and managing the notion of
> > > stream
> > > time is valuable and not something we would want to delegate to the
> > > tasks. I'm not entirely convinced that a separate callback (option
> > > C)
> > > is that messy (it could just be a default method with an empty
> > > implementation), but if we wanted a single API to handle both
> > > cases,
> > > how about something like the following?
> > >
> > > enum Time {
> > >    STREAM,
> > >    CLOCK
> > > }
> > >
> > > Then on ProcessorContext:
> > > context.schedule(Time time, long interval)  // We could allow this
> > > to
> > > be called once for each value of time to mix approaches.
> > >
> > > Then the Processor API becomes:
> > > punctuate(Time time) // time here denotes which schedule resulted
> > > in
> > > this call.
> > >
> > > Thoughts?
> > >
> > >
> > > On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
> > > >
> > > > Thanks a lot for the KIP Michal,
> > > >
> > > > I was thinking about the four options you proposed in more
> > > > details
> > > > and
> > > > this are my thoughts:
> > > >
> > > > (A) You argue, that users can still "punctuate" on event-time via
> > > > process(), but I am not sure if this is possible. Note, that
> > > > users
> > > > only
> > > > get record timestamps via context.timestamp(). Thus, users would
> > > > need
> > > > to
> > > > track the time progress per partition (based on the partitions
> > > > they
> > > > obverse via context.partition(). (This alone puts a huge burden
> > > > on
> > > > the
> > > > user by itself.) However, users are not notified at startup what
> > > > partitions are assigned, and user are not notified when
> > > > partitions
> > > > get
> > > > revoked. Because this information is not available, it's not
> > > > possible
> > > > to
> > > > "manually advance" stream-time, and thus event-time punctuation
> > > > within
> > > > process() seems not to be possible -- or do you see a way to get
> > > > it
> > > > done? And even if, it might still be too clumsy to use.
> > > >
> > > > (B) This does not allow to mix both approaches, thus limiting
> > > > what
> > > > users
> > > > can do.
> > > >
> > > > (C) This should give all flexibility we need. However, just
> > > > adding
> > > > one
> > > > more method seems to be a solution that is too simple (cf my
> > > > comments
> > > > below).
> > > >
> > > > (D) This might be hard to use. Also, I am not sure how a user
> > > > could
> > > > enable system-time and event-time punctuation in parallel.
> > > >
> > > >
> > > >
> > > > Overall options (C) seems to be the most promising approach to
> > > > me.
> > > > Because I also favor a clean API, we might keep current
> > > > punctuate()
> > > > as-is, but deprecate it -- so we can remove it at some later
> > > > point
> > > > when
> > > > people use the "new punctuate API".
> > > >
> > > >
> > > > Couple of follow up questions:
> > > >
> > > > - I am wondering, if we should have two callback methods or just
> > > > one
> > > > (ie, a unified for system and event time punctuation or one for
> > > > each?).
> > > >
> > > > - If we have one, how can the user figure out, which condition
> > > > did
> > > > trigger?
> > > >
> > > > - How would the API look like, for registering different
> > > > punctuate
> > > > schedules? The "type" must be somehow defined?
> > > >
> > > > - We might want to add "complex" schedules later on (like,
> > > > punctuate
> > > > on
> > > > every 10 seconds event-time or 60 seconds system-time whatever
> > > > comes
> > > > first). I don't say we should add this right away, but we might
> > > > want
> > > > to
> > > > define the API in a way, that it allows extensions like this
> > > > later
> > > > on,
> > > > without redesigning the API (ie, the API should be designed
> > > > extensible)
> > > >
> > > > - Did you ever consider count-based punctuation?
> > > >
> > > >
> > > > I understand, that you would like to solve a simple problem, but
> > > > we
> > > > learned from the past, that just "adding some API" quickly leads
> > > > to a
> > > > not very well defined API that needs time consuming clean up
> > > > later on
> > > > via other KIPs. Thus, I would prefer to get a holistic
> > > > punctuation
> > > > KIP
> > > > with this from the beginning on to avoid later painful redesign.
> > > >
> > > >
> > > >
> > > > -Matthias
> > > >
> > > >
> > > >
> > > > On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> > > > >
> > > > >
> > > > > Thanks Thomas,
> > > > >
> > > > > I'm also wary of changing the existing semantics of punctuate,
> > > > > for
> > > > > backward compatibility reasons, although I like the conceptual
> > > > > simplicity of that option.
> > > > >
> > > > > Adding a new method to me feels safer but, in a way, uglier. I
> > > > > added
> > > > > this to the KIP now as option (C).
> > > > >
> > > > > The TimestampExtractor mechanism is actually more flexible, as
> > > > > it
> > > > > allows
> > > > > you to return any value, you're not limited to event time or
> > > > > system
> > > > > time
> > > > > (although I don't see an actual use case where you might need
> > > > > anything
> > > > > else then those two). Hence I also proposed the option to allow
> > > > > users
> > > > > to, effectively, decide what "stream time" is for them given
> > > > > the
> > > > > presence or absence of messages, much like they can decide what
> > > > > msg
> > > > > time
> > > > > means for them using the TimestampExtractor. What do you think
> > > > > about
> > > > > that? This is probably most flexible but also most complicated.
> > > > >
> > > > > All comments appreciated.
> > > > >
> > > > > Cheers,
> > > > >
> > > > > Michal
> > > > >
> > > > >
> > > > > On 03/04/17 19:23, Thomas Becker wrote:
> > > > > >
> > > > > >
> > > > > > Although I fully agree we need a way to trigger periodic
> > > > > > processing
> > > > > > that is independent from whether and when messages arrive,
> > > > > > I'm
> > > > > > not sure
> > > > > > I like the idea of changing the existing semantics across the
> > > > > > board.
> > > > > > What if we added an additional callback to Processor that can
> > > > > > be
> > > > > > scheduled similarly to punctuate() but was always called at
> > > > > > fixed, wall
> > > > > > clock based intervals? This way you wouldn't have to give up
> > > > > > the
> > > > > > notion
> > > > > > of stream time to be able to do periodic processing.
> > > > > >
> > > > > > On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
> > > > > > >
> > > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > I have created a draft for KIP-138: Change punctuate
> > > > > > > semantics
> > > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%
> > > > > > > 3A+C
> > > > > > > hange+
> > > > > > > punctuate+semantics>
> > > > > > > .
> > > > > > >
> > > > > > > Appreciating there can be different views on system-time vs
> > > > > > > event-
> > > > > > > time
> > > > > > > semantics for punctuation depending on use-case and the
> > > > > > > importance of
> > > > > > > backwards compatibility of any such change, I've left it
> > > > > > > quite
> > > > > > > open
> > > > > > > and
> > > > > > > hope to fill in more info as the discussion progresses.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Michal
> > > --
> > >
> > >
> > >     Tommy Becker
> > >
> > >     Senior Software Engineer
> > >
> > >     O +1 919.460.4747
> > >
> > >     tivo.com
> > >
> > >
> > > ________________________________
> > >
> > > This email and any attachments may contain confidential and
> > > privileged material for the sole use of the intended recipient. Any
> > > review, copying, or distribution of this email (or any attachments)
> > > by others is prohibited. If you are not the intended recipient,
> > > please contact the sender immediately and permanently delete this
> > > email and any attachments. No employee or agent of TiVo Inc. is
> > > authorized to conclude any binding agreement on behalf of TiVo Inc.
> > > by email. Binding agreements with TiVo Inc. may only be made by a
> > > signed written agreement.
> > >
> --
>
>
>     Tommy Becker
>
>     Senior Software Engineer
>
>     O +1 919.460.4747
>
>     tivo.com
>
>
> ________________________________
>
> This email and any attachments may contain confidential and privileged
> material for the sole use of the intended recipient. Any review, copying,
> or distribution of this email (or any attachments) by others is prohibited.
> If you are not the intended recipient, please contact the sender
> immediately and permanently delete this email and any attachments. No
> employee or agent of TiVo Inc. is authorized to conclude any binding
> agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> Inc. may only be made by a signed written agreement.
>

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Thomas Becker <to...@Tivo.com>.
Yeah I like PuncutationType much better; I just threw Time out there
more as a strawman than an actual suggestion ;) I still think it's
worth considering what this buys us over an additional callback. I
foresee a number of punctuate implementations following this pattern:

public void punctuate(PunctuationType type) {
    switch (type) {
        case EVENT_TIME:
            methodA();
            break;
        case SYSTEM_TIME:
            methodB();
            break;
    }
}

I guess one advantage of this approach is we could add additional
punctuation types later in a backwards compatible way (like event count
as you mentioned).

-Tommy


On Tue, 2017-04-04 at 11:10 -0700, Matthias J. Sax wrote:
> That sounds promising.
>
> I am just wondering if `Time` is the best name. Maybe we want to add
> other non-time based punctuations at some point later. I would
> suggest
>
> enum PunctuationType {
>   EVENT_TIME,
>   SYSTEM_TIME,
> }
>
> or similar. Just to keep the door open -- it's easier to add new
> stuff
> if the name is more generic.
>
>
> -Matthias
>
>
> On 4/4/17 5:30 AM, Thomas Becker wrote:
> >
> > I agree that the framework providing and managing the notion of
> > stream
> > time is valuable and not something we would want to delegate to the
> > tasks. I'm not entirely convinced that a separate callback (option
> > C)
> > is that messy (it could just be a default method with an empty
> > implementation), but if we wanted a single API to handle both
> > cases,
> > how about something like the following?
> >
> > enum Time {
> >    STREAM,
> >    CLOCK
> > }
> >
> > Then on ProcessorContext:
> > context.schedule(Time time, long interval)  // We could allow this
> > to
> > be called once for each value of time to mix approaches.
> >
> > Then the Processor API becomes:
> > punctuate(Time time) // time here denotes which schedule resulted
> > in
> > this call.
> >
> > Thoughts?
> >
> >
> > On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
> > >
> > > Thanks a lot for the KIP Michal,
> > >
> > > I was thinking about the four options you proposed in more
> > > details
> > > and
> > > this are my thoughts:
> > >
> > > (A) You argue, that users can still "punctuate" on event-time via
> > > process(), but I am not sure if this is possible. Note, that
> > > users
> > > only
> > > get record timestamps via context.timestamp(). Thus, users would
> > > need
> > > to
> > > track the time progress per partition (based on the partitions
> > > they
> > > obverse via context.partition(). (This alone puts a huge burden
> > > on
> > > the
> > > user by itself.) However, users are not notified at startup what
> > > partitions are assigned, and user are not notified when
> > > partitions
> > > get
> > > revoked. Because this information is not available, it's not
> > > possible
> > > to
> > > "manually advance" stream-time, and thus event-time punctuation
> > > within
> > > process() seems not to be possible -- or do you see a way to get
> > > it
> > > done? And even if, it might still be too clumsy to use.
> > >
> > > (B) This does not allow to mix both approaches, thus limiting
> > > what
> > > users
> > > can do.
> > >
> > > (C) This should give all flexibility we need. However, just
> > > adding
> > > one
> > > more method seems to be a solution that is too simple (cf my
> > > comments
> > > below).
> > >
> > > (D) This might be hard to use. Also, I am not sure how a user
> > > could
> > > enable system-time and event-time punctuation in parallel.
> > >
> > >
> > >
> > > Overall options (C) seems to be the most promising approach to
> > > me.
> > > Because I also favor a clean API, we might keep current
> > > punctuate()
> > > as-is, but deprecate it -- so we can remove it at some later
> > > point
> > > when
> > > people use the "new punctuate API".
> > >
> > >
> > > Couple of follow up questions:
> > >
> > > - I am wondering, if we should have two callback methods or just
> > > one
> > > (ie, a unified for system and event time punctuation or one for
> > > each?).
> > >
> > > - If we have one, how can the user figure out, which condition
> > > did
> > > trigger?
> > >
> > > - How would the API look like, for registering different
> > > punctuate
> > > schedules? The "type" must be somehow defined?
> > >
> > > - We might want to add "complex" schedules later on (like,
> > > punctuate
> > > on
> > > every 10 seconds event-time or 60 seconds system-time whatever
> > > comes
> > > first). I don't say we should add this right away, but we might
> > > want
> > > to
> > > define the API in a way, that it allows extensions like this
> > > later
> > > on,
> > > without redesigning the API (ie, the API should be designed
> > > extensible)
> > >
> > > - Did you ever consider count-based punctuation?
> > >
> > >
> > > I understand, that you would like to solve a simple problem, but
> > > we
> > > learned from the past, that just "adding some API" quickly leads
> > > to a
> > > not very well defined API that needs time consuming clean up
> > > later on
> > > via other KIPs. Thus, I would prefer to get a holistic
> > > punctuation
> > > KIP
> > > with this from the beginning on to avoid later painful redesign.
> > >
> > >
> > >
> > > -Matthias
> > >
> > >
> > >
> > > On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> > > >
> > > >
> > > > Thanks Thomas,
> > > >
> > > > I'm also wary of changing the existing semantics of punctuate,
> > > > for
> > > > backward compatibility reasons, although I like the conceptual
> > > > simplicity of that option.
> > > >
> > > > Adding a new method to me feels safer but, in a way, uglier. I
> > > > added
> > > > this to the KIP now as option (C).
> > > >
> > > > The TimestampExtractor mechanism is actually more flexible, as
> > > > it
> > > > allows
> > > > you to return any value, you're not limited to event time or
> > > > system
> > > > time
> > > > (although I don't see an actual use case where you might need
> > > > anything
> > > > else then those two). Hence I also proposed the option to allow
> > > > users
> > > > to, effectively, decide what "stream time" is for them given
> > > > the
> > > > presence or absence of messages, much like they can decide what
> > > > msg
> > > > time
> > > > means for them using the TimestampExtractor. What do you think
> > > > about
> > > > that? This is probably most flexible but also most complicated.
> > > >
> > > > All comments appreciated.
> > > >
> > > > Cheers,
> > > >
> > > > Michal
> > > >
> > > >
> > > > On 03/04/17 19:23, Thomas Becker wrote:
> > > > >
> > > > >
> > > > > Although I fully agree we need a way to trigger periodic
> > > > > processing
> > > > > that is independent from whether and when messages arrive,
> > > > > I'm
> > > > > not sure
> > > > > I like the idea of changing the existing semantics across the
> > > > > board.
> > > > > What if we added an additional callback to Processor that can
> > > > > be
> > > > > scheduled similarly to punctuate() but was always called at
> > > > > fixed, wall
> > > > > clock based intervals? This way you wouldn't have to give up
> > > > > the
> > > > > notion
> > > > > of stream time to be able to do periodic processing.
> > > > >
> > > > > On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
> > > > > >
> > > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > I have created a draft for KIP-138: Change punctuate
> > > > > > semantics
> > > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%
> > > > > > 3A+C
> > > > > > hange+
> > > > > > punctuate+semantics>
> > > > > > .
> > > > > >
> > > > > > Appreciating there can be different views on system-time vs
> > > > > > event-
> > > > > > time
> > > > > > semantics for punctuation depending on use-case and the
> > > > > > importance of
> > > > > > backwards compatibility of any such change, I've left it
> > > > > > quite
> > > > > > open
> > > > > > and
> > > > > > hope to fill in more info as the discussion progresses.
> > > > > >
> > > > > > Thanks,
> > > > > > Michal
> > --
> >
> >
> >     Tommy Becker
> >
> >     Senior Software Engineer
> >
> >     O +1 919.460.4747
> >
> >     tivo.com
> >
> >
> > ________________________________
> >
> > This email and any attachments may contain confidential and
> > privileged material for the sole use of the intended recipient. Any
> > review, copying, or distribution of this email (or any attachments)
> > by others is prohibited. If you are not the intended recipient,
> > please contact the sender immediately and permanently delete this
> > email and any attachments. No employee or agent of TiVo Inc. is
> > authorized to conclude any binding agreement on behalf of TiVo Inc.
> > by email. Binding agreements with TiVo Inc. may only be made by a
> > signed written agreement.
> >
--


    Tommy Becker

    Senior Software Engineer

    O +1 919.460.4747

    tivo.com


________________________________

This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Sounds promising to me too.

I'll update the KIP with this as the primary proposal but leave the other
alternatives there as under consideration for now.

Cheers,
Michal

On 4 April 2017 at 19:10, Matthias J. Sax <ma...@confluent.io> wrote:

> That sounds promising.
>
> I am just wondering if `Time` is the best name. Maybe we want to add
> other non-time based punctuations at some point later. I would suggest
>
> enum PunctuationType {
>   EVENT_TIME,
>   SYSTEM_TIME,
> }
>
> or similar. Just to keep the door open -- it's easier to add new stuff
> if the name is more generic.
>
>
> -Matthias
>
>
> On 4/4/17 5:30 AM, Thomas Becker wrote:
> > I agree that the framework providing and managing the notion of stream
> > time is valuable and not something we would want to delegate to the
> > tasks. I'm not entirely convinced that a separate callback (option C)
> > is that messy (it could just be a default method with an empty
> > implementation), but if we wanted a single API to handle both cases,
> > how about something like the following?
> >
> > enum Time {
> >    STREAM,
> >    CLOCK
> > }
> >
> > Then on ProcessorContext:
> > context.schedule(Time time, long interval)  // We could allow this to
> > be called once for each value of time to mix approaches.
> >
> > Then the Processor API becomes:
> > punctuate(Time time) // time here denotes which schedule resulted in
> > this call.
> >
> > Thoughts?
> >
> >
> > On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
> >> Thanks a lot for the KIP Michal,
> >>
> >> I was thinking about the four options you proposed in more details
> >> and
> >> this are my thoughts:
> >>
> >> (A) You argue, that users can still "punctuate" on event-time via
> >> process(), but I am not sure if this is possible. Note, that users
> >> only
> >> get record timestamps via context.timestamp(). Thus, users would need
> >> to
> >> track the time progress per partition (based on the partitions they
> >> obverse via context.partition(). (This alone puts a huge burden on
> >> the
> >> user by itself.) However, users are not notified at startup what
> >> partitions are assigned, and user are not notified when partitions
> >> get
> >> revoked. Because this information is not available, it's not possible
> >> to
> >> "manually advance" stream-time, and thus event-time punctuation
> >> within
> >> process() seems not to be possible -- or do you see a way to get it
> >> done? And even if, it might still be too clumsy to use.
> >>
> >> (B) This does not allow to mix both approaches, thus limiting what
> >> users
> >> can do.
> >>
> >> (C) This should give all flexibility we need. However, just adding
> >> one
> >> more method seems to be a solution that is too simple (cf my comments
> >> below).
> >>
> >> (D) This might be hard to use. Also, I am not sure how a user could
> >> enable system-time and event-time punctuation in parallel.
> >>
> >>
> >>
> >> Overall options (C) seems to be the most promising approach to me.
> >> Because I also favor a clean API, we might keep current punctuate()
> >> as-is, but deprecate it -- so we can remove it at some later point
> >> when
> >> people use the "new punctuate API".
> >>
> >>
> >> Couple of follow up questions:
> >>
> >> - I am wondering, if we should have two callback methods or just one
> >> (ie, a unified for system and event time punctuation or one for
> >> each?).
> >>
> >> - If we have one, how can the user figure out, which condition did
> >> trigger?
> >>
> >> - How would the API look like, for registering different punctuate
> >> schedules? The "type" must be somehow defined?
> >>
> >> - We might want to add "complex" schedules later on (like, punctuate
> >> on
> >> every 10 seconds event-time or 60 seconds system-time whatever comes
> >> first). I don't say we should add this right away, but we might want
> >> to
> >> define the API in a way, that it allows extensions like this later
> >> on,
> >> without redesigning the API (ie, the API should be designed
> >> extensible)
> >>
> >> - Did you ever consider count-based punctuation?
> >>
> >>
> >> I understand, that you would like to solve a simple problem, but we
> >> learned from the past, that just "adding some API" quickly leads to a
> >> not very well defined API that needs time consuming clean up later on
> >> via other KIPs. Thus, I would prefer to get a holistic punctuation
> >> KIP
> >> with this from the beginning on to avoid later painful redesign.
> >>
> >>
> >>
> >> -Matthias
> >>
> >>
> >>
> >> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> >>>
> >>> Thanks Thomas,
> >>>
> >>> I'm also wary of changing the existing semantics of punctuate, for
> >>> backward compatibility reasons, although I like the conceptual
> >>> simplicity of that option.
> >>>
> >>> Adding a new method to me feels safer but, in a way, uglier. I
> >>> added
> >>> this to the KIP now as option (C).
> >>>
> >>> The TimestampExtractor mechanism is actually more flexible, as it
> >>> allows
> >>> you to return any value, you're not limited to event time or system
> >>> time
> >>> (although I don't see an actual use case where you might need
> >>> anything
> >>> else then those two). Hence I also proposed the option to allow
> >>> users
> >>> to, effectively, decide what "stream time" is for them given the
> >>> presence or absence of messages, much like they can decide what msg
> >>> time
> >>> means for them using the TimestampExtractor. What do you think
> >>> about
> >>> that? This is probably most flexible but also most complicated.
> >>>
> >>> All comments appreciated.
> >>>
> >>> Cheers,
> >>>
> >>> Michal
> >>>
> >>>
> >>> On 03/04/17 19:23, Thomas Becker wrote:
> >>>>
> >>>> Although I fully agree we need a way to trigger periodic
> >>>> processing
> >>>> that is independent from whether and when messages arrive, I'm
> >>>> not sure
> >>>> I like the idea of changing the existing semantics across the
> >>>> board.
> >>>> What if we added an additional callback to Processor that can be
> >>>> scheduled similarly to punctuate() but was always called at
> >>>> fixed, wall
> >>>> clock based intervals? This way you wouldn't have to give up the
> >>>> notion
> >>>> of stream time to be able to do periodic processing.
> >>>>
> >>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
> >>>>>
> >>>>> Hi all,
> >>>>>
> >>>>> I have created a draft for KIP-138: Change punctuate semantics
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%3A+C
> >>>>> hange+
> >>>>> punctuate+semantics>
> >>>>> .
> >>>>>
> >>>>> Appreciating there can be different views on system-time vs
> >>>>> event-
> >>>>> time
> >>>>> semantics for punctuation depending on use-case and the
> >>>>> importance of
> >>>>> backwards compatibility of any such change, I've left it quite
> >>>>> open
> >>>>> and
> >>>>> hope to fill in more info as the discussion progresses.
> >>>>>
> >>>>> Thanks,
> >>>>> Michal
> > --
> >
> >
> >     Tommy Becker
> >
> >     Senior Software Engineer
> >
> >     O +1 919.460.4747
> >
> >     tivo.com
> >
> >
> > ________________________________
> >
> > This email and any attachments may contain confidential and privileged
> material for the sole use of the intended recipient. Any review, copying,
> or distribution of this email (or any attachments) by others is prohibited.
> If you are not the intended recipient, please contact the sender
> immediately and permanently delete this email and any attachments. No
> employee or agent of TiVo Inc. is authorized to conclude any binding
> agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> Inc. may only be made by a signed written agreement.
> >
>
>


-- 
<http://www.openbet.com> Michal Borowiecki
Technical Lead
T: +44 208 742 1600


+44 203 249 8448



E: michal.borowiecki@openbet.com
W: www.openbet.com
OpenBet Ltd

Chiswick Park Building 9

566 Chiswick High Rd

London

W4 5XT <http://twitter.com/OpenBet_Ltd>
<http://www.linkedin.com/company/165331/> <http://www.facebook.com/OpenBet>
Winner of Sports Betting Supplier of the Year at the EGR B2B Awards 2010,
2011 & 2012
This message is confidential and intended only for the addressee. If you
have received this message in error, please immediately notify the
postmaster@openbet.com and delete it from your system as well as any
copies. The content of e-mails as well as traffic data may be monitored by
OpenBet for employment and security purposes. To protect the environment
please do not print this e-mail unless necessary. OpenBet Ltd. Registered
Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
United Kingdom. A company registered in England and Wales. Registered no.
3134634. VAT no. GB927523612

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
That sounds promising.

I am just wondering if `Time` is the best name. Maybe we want to add
other non-time based punctuations at some point later. I would suggest

enum PunctuationType {
  EVENT_TIME,
  SYSTEM_TIME,
}

or similar. Just to keep the door open -- it's easier to add new stuff
if the name is more generic.


-Matthias


On 4/4/17 5:30 AM, Thomas Becker wrote:
> I agree that the framework providing and managing the notion of stream
> time is valuable and not something we would want to delegate to the
> tasks. I'm not entirely convinced that a separate callback (option C)
> is that messy (it could just be a default method with an empty
> implementation), but if we wanted a single API to handle both cases,
> how about something like the following?
> 
> enum Time {
>    STREAM,
>    CLOCK
> }
> 
> Then on ProcessorContext:
> context.schedule(Time time, long interval)  // We could allow this to
> be called once for each value of time to mix approaches.
> 
> Then the Processor API becomes:
> punctuate(Time time) // time here denotes which schedule resulted in
> this call.
> 
> Thoughts?
> 
> 
> On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
>> Thanks a lot for the KIP Michal,
>>
>> I was thinking about the four options you proposed in more details
>> and
>> this are my thoughts:
>>
>> (A) You argue, that users can still "punctuate" on event-time via
>> process(), but I am not sure if this is possible. Note, that users
>> only
>> get record timestamps via context.timestamp(). Thus, users would need
>> to
>> track the time progress per partition (based on the partitions they
>> obverse via context.partition(). (This alone puts a huge burden on
>> the
>> user by itself.) However, users are not notified at startup what
>> partitions are assigned, and user are not notified when partitions
>> get
>> revoked. Because this information is not available, it's not possible
>> to
>> "manually advance" stream-time, and thus event-time punctuation
>> within
>> process() seems not to be possible -- or do you see a way to get it
>> done? And even if, it might still be too clumsy to use.
>>
>> (B) This does not allow to mix both approaches, thus limiting what
>> users
>> can do.
>>
>> (C) This should give all flexibility we need. However, just adding
>> one
>> more method seems to be a solution that is too simple (cf my comments
>> below).
>>
>> (D) This might be hard to use. Also, I am not sure how a user could
>> enable system-time and event-time punctuation in parallel.
>>
>>
>>
>> Overall options (C) seems to be the most promising approach to me.
>> Because I also favor a clean API, we might keep current punctuate()
>> as-is, but deprecate it -- so we can remove it at some later point
>> when
>> people use the "new punctuate API".
>>
>>
>> Couple of follow up questions:
>>
>> - I am wondering, if we should have two callback methods or just one
>> (ie, a unified for system and event time punctuation or one for
>> each?).
>>
>> - If we have one, how can the user figure out, which condition did
>> trigger?
>>
>> - How would the API look like, for registering different punctuate
>> schedules? The "type" must be somehow defined?
>>
>> - We might want to add "complex" schedules later on (like, punctuate
>> on
>> every 10 seconds event-time or 60 seconds system-time whatever comes
>> first). I don't say we should add this right away, but we might want
>> to
>> define the API in a way, that it allows extensions like this later
>> on,
>> without redesigning the API (ie, the API should be designed
>> extensible)
>>
>> - Did you ever consider count-based punctuation?
>>
>>
>> I understand, that you would like to solve a simple problem, but we
>> learned from the past, that just "adding some API" quickly leads to a
>> not very well defined API that needs time consuming clean up later on
>> via other KIPs. Thus, I would prefer to get a holistic punctuation
>> KIP
>> with this from the beginning on to avoid later painful redesign.
>>
>>
>>
>> -Matthias
>>
>>
>>
>> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
>>>
>>> Thanks Thomas,
>>>
>>> I'm also wary of changing the existing semantics of punctuate, for
>>> backward compatibility reasons, although I like the conceptual
>>> simplicity of that option.
>>>
>>> Adding a new method to me feels safer but, in a way, uglier. I
>>> added
>>> this to the KIP now as option (C).
>>>
>>> The TimestampExtractor mechanism is actually more flexible, as it
>>> allows
>>> you to return any value, you're not limited to event time or system
>>> time
>>> (although I don't see an actual use case where you might need
>>> anything
>>> else then those two). Hence I also proposed the option to allow
>>> users
>>> to, effectively, decide what "stream time" is for them given the
>>> presence or absence of messages, much like they can decide what msg
>>> time
>>> means for them using the TimestampExtractor. What do you think
>>> about
>>> that? This is probably most flexible but also most complicated.
>>>
>>> All comments appreciated.
>>>
>>> Cheers,
>>>
>>> Michal
>>>
>>>
>>> On 03/04/17 19:23, Thomas Becker wrote:
>>>>
>>>> Although I fully agree we need a way to trigger periodic
>>>> processing
>>>> that is independent from whether and when messages arrive, I'm
>>>> not sure
>>>> I like the idea of changing the existing semantics across the
>>>> board.
>>>> What if we added an additional callback to Processor that can be
>>>> scheduled similarly to punctuate() but was always called at
>>>> fixed, wall
>>>> clock based intervals? This way you wouldn't have to give up the
>>>> notion
>>>> of stream time to be able to do periodic processing.
>>>>
>>>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>>>>>
>>>>> Hi all,
>>>>>
>>>>> I have created a draft for KIP-138: Change punctuate semantics
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%3A+C
>>>>> hange+
>>>>> punctuate+semantics>
>>>>> .
>>>>>
>>>>> Appreciating there can be different views on system-time vs
>>>>> event-
>>>>> time
>>>>> semantics for punctuation depending on use-case and the
>>>>> importance of
>>>>> backwards compatibility of any such change, I've left it quite
>>>>> open
>>>>> and
>>>>> hope to fill in more info as the discussion progresses.
>>>>>
>>>>> Thanks,
>>>>> Michal
> --
> 
> 
>     Tommy Becker
> 
>     Senior Software Engineer
> 
>     O +1 919.460.4747
> 
>     tivo.com
> 
> 
> ________________________________
> 
> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Thomas Becker <to...@Tivo.com>.
I agree that the framework providing and managing the notion of stream
time is valuable and not something we would want to delegate to the
tasks. I'm not entirely convinced that a separate callback (option C)
is that messy (it could just be a default method with an empty
implementation), but if we wanted a single API to handle both cases,
how about something like the following?

enum Time {
   STREAM,
   CLOCK
}

Then on ProcessorContext:
context.schedule(Time time, long interval)  // We could allow this to
be called once for each value of time to mix approaches.

Then the Processor API becomes:
punctuate(Time time) // time here denotes which schedule resulted in
this call.

Thoughts?


On Mon, 2017-04-03 at 22:44 -0700, Matthias J. Sax wrote:
> Thanks a lot for the KIP Michal,
>
> I was thinking about the four options you proposed in more details
> and
> this are my thoughts:
>
> (A) You argue, that users can still "punctuate" on event-time via
> process(), but I am not sure if this is possible. Note, that users
> only
> get record timestamps via context.timestamp(). Thus, users would need
> to
> track the time progress per partition (based on the partitions they
> obverse via context.partition(). (This alone puts a huge burden on
> the
> user by itself.) However, users are not notified at startup what
> partitions are assigned, and user are not notified when partitions
> get
> revoked. Because this information is not available, it's not possible
> to
> "manually advance" stream-time, and thus event-time punctuation
> within
> process() seems not to be possible -- or do you see a way to get it
> done? And even if, it might still be too clumsy to use.
>
> (B) This does not allow to mix both approaches, thus limiting what
> users
> can do.
>
> (C) This should give all flexibility we need. However, just adding
> one
> more method seems to be a solution that is too simple (cf my comments
> below).
>
> (D) This might be hard to use. Also, I am not sure how a user could
> enable system-time and event-time punctuation in parallel.
>
>
>
> Overall options (C) seems to be the most promising approach to me.
> Because I also favor a clean API, we might keep current punctuate()
> as-is, but deprecate it -- so we can remove it at some later point
> when
> people use the "new punctuate API".
>
>
> Couple of follow up questions:
>
> - I am wondering, if we should have two callback methods or just one
> (ie, a unified for system and event time punctuation or one for
> each?).
>
> - If we have one, how can the user figure out, which condition did
> trigger?
>
> - How would the API look like, for registering different punctuate
> schedules? The "type" must be somehow defined?
>
> - We might want to add "complex" schedules later on (like, punctuate
> on
> every 10 seconds event-time or 60 seconds system-time whatever comes
> first). I don't say we should add this right away, but we might want
> to
> define the API in a way, that it allows extensions like this later
> on,
> without redesigning the API (ie, the API should be designed
> extensible)
>
> - Did you ever consider count-based punctuation?
>
>
> I understand, that you would like to solve a simple problem, but we
> learned from the past, that just "adding some API" quickly leads to a
> not very well defined API that needs time consuming clean up later on
> via other KIPs. Thus, I would prefer to get a holistic punctuation
> KIP
> with this from the beginning on to avoid later painful redesign.
>
>
>
> -Matthias
>
>
>
> On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> >
> > Thanks Thomas,
> >
> > I'm also wary of changing the existing semantics of punctuate, for
> > backward compatibility reasons, although I like the conceptual
> > simplicity of that option.
> >
> > Adding a new method to me feels safer but, in a way, uglier. I
> > added
> > this to the KIP now as option (C).
> >
> > The TimestampExtractor mechanism is actually more flexible, as it
> > allows
> > you to return any value, you're not limited to event time or system
> > time
> > (although I don't see an actual use case where you might need
> > anything
> > else then those two). Hence I also proposed the option to allow
> > users
> > to, effectively, decide what "stream time" is for them given the
> > presence or absence of messages, much like they can decide what msg
> > time
> > means for them using the TimestampExtractor. What do you think
> > about
> > that? This is probably most flexible but also most complicated.
> >
> > All comments appreciated.
> >
> > Cheers,
> >
> > Michal
> >
> >
> > On 03/04/17 19:23, Thomas Becker wrote:
> > >
> > > Although I fully agree we need a way to trigger periodic
> > > processing
> > > that is independent from whether and when messages arrive, I'm
> > > not sure
> > > I like the idea of changing the existing semantics across the
> > > board.
> > > What if we added an additional callback to Processor that can be
> > > scheduled similarly to punctuate() but was always called at
> > > fixed, wall
> > > clock based intervals? This way you wouldn't have to give up the
> > > notion
> > > of stream time to be able to do periodic processing.
> > >
> > > On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
> > > >
> > > > Hi all,
> > > >
> > > > I have created a draft for KIP-138: Change punctuate semantics
> > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%3A+C
> > > > hange+
> > > > punctuate+semantics>
> > > > .
> > > >
> > > > Appreciating there can be different views on system-time vs
> > > > event-
> > > > time
> > > > semantics for punctuation depending on use-case and the
> > > > importance of
> > > > backwards compatibility of any such change, I've left it quite
> > > > open
> > > > and
> > > > hope to fill in more info as the discussion progresses.
> > > >
> > > > Thanks,
> > > > Michal
--


    Tommy Becker

    Senior Software Engineer

    O +1 919.460.4747

    tivo.com


________________________________

This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Thanks a lot for the KIP Michal,

I was thinking about the four options you proposed in more details and
this are my thoughts:

(A) You argue, that users can still "punctuate" on event-time via
process(), but I am not sure if this is possible. Note, that users only
get record timestamps via context.timestamp(). Thus, users would need to
track the time progress per partition (based on the partitions they
obverse via context.partition(). (This alone puts a huge burden on the
user by itself.) However, users are not notified at startup what
partitions are assigned, and user are not notified when partitions get
revoked. Because this information is not available, it's not possible to
"manually advance" stream-time, and thus event-time punctuation within
process() seems not to be possible -- or do you see a way to get it
done? And even if, it might still be too clumsy to use.

(B) This does not allow to mix both approaches, thus limiting what users
can do.

(C) This should give all flexibility we need. However, just adding one
more method seems to be a solution that is too simple (cf my comments
below).

(D) This might be hard to use. Also, I am not sure how a user could
enable system-time and event-time punctuation in parallel.



Overall options (C) seems to be the most promising approach to me.
Because I also favor a clean API, we might keep current punctuate()
as-is, but deprecate it -- so we can remove it at some later point when
people use the "new punctuate API".


Couple of follow up questions:

- I am wondering, if we should have two callback methods or just one
(ie, a unified for system and event time punctuation or one for each?).

- If we have one, how can the user figure out, which condition did trigger?

- How would the API look like, for registering different punctuate
schedules? The "type" must be somehow defined?

- We might want to add "complex" schedules later on (like, punctuate on
every 10 seconds event-time or 60 seconds system-time whatever comes
first). I don't say we should add this right away, but we might want to
define the API in a way, that it allows extensions like this later on,
without redesigning the API (ie, the API should be designed extensible)

- Did you ever consider count-based punctuation?


I understand, that you would like to solve a simple problem, but we
learned from the past, that just "adding some API" quickly leads to a
not very well defined API that needs time consuming clean up later on
via other KIPs. Thus, I would prefer to get a holistic punctuation KIP
with this from the beginning on to avoid later painful redesign.



-Matthias



On 4/3/17 11:58 AM, Michal Borowiecki wrote:
> Thanks Thomas,
> 
> I'm also wary of changing the existing semantics of punctuate, for
> backward compatibility reasons, although I like the conceptual
> simplicity of that option.
> 
> Adding a new method to me feels safer but, in a way, uglier. I added
> this to the KIP now as option (C).
> 
> The TimestampExtractor mechanism is actually more flexible, as it allows
> you to return any value, you're not limited to event time or system time
> (although I don't see an actual use case where you might need anything
> else then those two). Hence I also proposed the option to allow users
> to, effectively, decide what "stream time" is for them given the
> presence or absence of messages, much like they can decide what msg time
> means for them using the TimestampExtractor. What do you think about
> that? This is probably most flexible but also most complicated.
> 
> All comments appreciated.
> 
> Cheers,
> 
> Michal
> 
> 
> On 03/04/17 19:23, Thomas Becker wrote:
>> Although I fully agree we need a way to trigger periodic processing
>> that is independent from whether and when messages arrive, I'm not sure
>> I like the idea of changing the existing semantics across the board.
>> What if we added an additional callback to Processor that can be
>> scheduled similarly to punctuate() but was always called at fixed, wall
>> clock based intervals? This way you wouldn't have to give up the notion
>> of stream time to be able to do periodic processing.
>>
>> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>>> Hi all,
>>>
>>> I have created a draft for KIP-138: Change punctuate semantics
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%3A+Change+
>>> punctuate+semantics>
>>> .
>>>
>>> Appreciating there can be different views on system-time vs event-
>>> time
>>> semantics for punctuation depending on use-case and the importance of
>>> backwards compatibility of any such change, I've left it quite open
>>> and
>>> hope to fill in more info as the discussion progresses.
>>>
>>> Thanks,
>>> Michal
>> -- 
>>
>>
>>      Tommy Becker
>>
>>      Senior Software Engineer
>>
>>      O +1 919.460.4747
>>
>>      tivo.com
>>
>>
>> ________________________________
>>
>> This email and any attachments may contain confidential and privileged
>> material for the sole use of the intended recipient. Any review,
>> copying, or distribution of this email (or any attachments) by others
>> is prohibited. If you are not the intended recipient, please contact
>> the sender immediately and permanently delete this email and any
>> attachments. No employee or agent of TiVo Inc. is authorized to
>> conclude any binding agreement on behalf of TiVo Inc. by email.
>> Binding agreements with TiVo Inc. may only be made by a signed written
>> agreement.
> 
> 


Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Michal Borowiecki <mi...@openbet.com>.
Thanks Thomas,

I'm also wary of changing the existing semantics of punctuate, for 
backward compatibility reasons, although I like the conceptual 
simplicity of that option.

Adding a new method to me feels safer but, in a way, uglier. I added 
this to the KIP now as option (C).

The TimestampExtractor mechanism is actually more flexible, as it allows 
you to return any value, you're not limited to event time or system time 
(although I don't see an actual use case where you might need anything 
else then those two). Hence I also proposed the option to allow users 
to, effectively, decide what "stream time" is for them given the 
presence or absence of messages, much like they can decide what msg time 
means for them using the TimestampExtractor. What do you think about 
that? This is probably most flexible but also most complicated.

All comments appreciated.

Cheers,

Michal


On 03/04/17 19:23, Thomas Becker wrote:
> Although I fully agree we need a way to trigger periodic processing
> that is independent from whether and when messages arrive, I'm not sure
> I like the idea of changing the existing semantics across the board.
> What if we added an additional callback to Processor that can be
> scheduled similarly to punctuate() but was always called at fixed, wall
> clock based intervals? This way you wouldn't have to give up the notion
> of stream time to be able to do periodic processing.
>
> On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
>> Hi all,
>>
>> I have created a draft for KIP-138: Change punctuate semantics
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%3A+Change+
>> punctuate+semantics>
>> .
>>
>> Appreciating there can be different views on system-time vs event-
>> time
>> semantics for punctuation depending on use-case and the importance of
>> backwards compatibility of any such change, I've left it quite open
>> and
>> hope to fill in more info as the discussion progresses.
>>
>> Thanks,
>> Michal
> --
>
>
>      Tommy Becker
>
>      Senior Software Engineer
>
>      O +1 919.460.4747
>
>      tivo.com
>
>
> ________________________________
>
> This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.



Re: [DISCUSS] KIP-138: Change punctuate semantics

Posted by Thomas Becker <to...@Tivo.com>.
Although I fully agree we need a way to trigger periodic processing
that is independent from whether and when messages arrive, I'm not sure
I like the idea of changing the existing semantics across the board.
What if we added an additional callback to Processor that can be
scheduled similarly to punctuate() but was always called at fixed, wall
clock based intervals? This way you wouldn't have to give up the notion
of stream time to be able to do periodic processing.

On Mon, 2017-04-03 at 10:34 +0100, Michal Borowiecki wrote:
> Hi all,
>
> I have created a draft for KIP-138: Change punctuate semantics
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%3A+Change+
> punctuate+semantics>
> .
>
> Appreciating there can be different views on system-time vs event-
> time
> semantics for punctuation depending on use-case and the importance of
> backwards compatibility of any such change, I've left it quite open
> and
> hope to fill in more info as the discussion progresses.
>
> Thanks,
> Michal
--


    Tommy Becker

    Senior Software Engineer

    O +1 919.460.4747

    tivo.com


________________________________

This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.