Hi John,

'The intermediate state of the suppression' in KIP does not mean the state
of upstream KTable - sure, the state of the upstream KTable can be queried
by materializing the operator immediately before the suppress as you shown.
What I meant in KIP was the final state of the buffer, which is not emitted
yet. (I agree, the current description may be confusing; it would be better
to change it with 'the current state of the suppression' or 'the results of
the suppression', like the Jira issue
<https://issues.apache.org/jira/browse/KAFKA-8403> states.)

For a little bit more about the motivation, here is one of my experience: I
had to build a monitoring application which collects signals from IoT
devices (say, a semiconductor production line.) If the number of collected
signals within the time window is much less than the expected, there may be
some problems like network hiccup in the systems. We wanted to build the
system in the form of a dashboard, but could not by lack of materializing
feature. It was precisely the case of querying only the final results of a
windowed aggregation, as the Jira issue
<https://issues.apache.org/jira/browse/KAFKA-8403> states. We finally ended
in implementing the system in an email alerting system like this
<https://www.confluent.io/blog/kafka-streams-take-on-watermarks-and-triggers/>
and had to collect the keys and windows of trouble by hand.

I think these kinds of use cases would be much common. Should it be
described in the KIP much more in detail?

Thanks,
Dongjin

On Sat, Feb 15, 2020 at 4:43 AM John Roesler <vvcep...@apache.org> wrote:

> Hi Dongjin,
>
> Thanks for the KIP!
>
> Can you explain more about why the internal data structures of suppression
> should be queriable? The motivation just says that users might want to do
> it, which seems like it could justify literally anything :)
>
> One design point of Suppression is that if you wanted to query the “final
> state”, you can Materialize the suppress itself (which is why it needs the
> variant); if you wanted to query the “intermediate state”, you can
> materialize the operator immediately before the suppress.
>
> Example:
>
> ...count(Materialized.as(“intermediate”))
>   .supress(untilWindowClosed(), Materialized.as(“final”))
>
> I’m not sure what use case would require actually fetching from the
> internal buffers.
>
> Thanks,
> John
>
>
> On Fri, Feb 14, 2020, at 07:55, Dongjin Lee wrote:
> > Hi devs,
> >
> > I'd like to reboot the discussion on KIP-508, which aims to support a
> > Materialized variant of KTable#suppress. It was initially submitted
> several
> > months ago but closed by the inactivity.
> >
> > - KIP:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-508%3A+Make+Suppression+State+Queriable
> > - Jira: https://issues.apache.org/jira/browse/KAFKA-8403
> >
> > All kinds of feedback will be greatly appreciated.
> >
> > Best,
> > Dongjin
> >
> > --
> > *Dongjin Lee*
> >
> > *A hitchhiker in the mathematical world.*
> > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > <https://github.com/dongjinleekr>linkedin:
> kr.linkedin.com/in/dongjinleekr
> > <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> speakerdeck.com/dongjin
> > <https://speakerdeck.com/dongjin>*
> >
>


-- 
*Dongjin Lee*

*A hitchhiker in the mathematical world.*
*github:  <http://goog_969573159/>github.com/dongjinleekr
<https://github.com/dongjinleekr>linkedin: kr.linkedin.com/in/dongjinleekr
<https://kr.linkedin.com/in/dongjinleekr>speakerdeck: speakerdeck.com/dongjin
<https://speakerdeck.com/dongjin>*

Reply via email to