You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Guozhang Wang <wa...@gmail.com> on 2017/05/11 05:02:32 UTC

[DISCUSS] Modify / Remove "Unstable" annotations in Streams API

Hello folks,

As we are approaching the feature freeze deadline of 0.11.0.0, one thing I
realized is that currently the Streams public APIs are still marked as
"Unstable", which is to indicate that the API itself does not provide
guarantees about backward compatibility across releases. On the other hand,
since Streams have now been widely adopted in production use cases by many
organizations, we are in fact evolving its APIs in a much stricter manner
than "Unstable" allows us: for all the current Streams related KIP
proposals under discussions right now [1], people have been working hard to
make sure none of them are going to break backward compatibility in the
coming releases. So I think it would be a good timing to change the Streams
API annotations.

My proposal would be the following:

1. For "o.a.k.streams.errors" and "o.a.k.streams.state" packages: remove
the annotations except `StreamsMetrics`.

2. For "o.a.k.streams.kstream": remove the annotations except "KStream",
"KTable", "GroupedKStream", "GroupedKTable", "GlobalKTable" and
"KStreamBuilder".

3. For all the other public classes, including "o.a.k.streams.processor"
and the above mentioned classes, change the annotation to "Evolving", which
means "we might break compatibility at minor releases (i.e. 0.12.x, 0.13.x,
1.0.x etc) only".


The ultimate goal is to make sure we won't break anything going forward,
hence in the future we should remove all the annotations to make that
clear. The above changes in 0.11.0.0 is to give us some "buffer time" in
case there are some major API change proposals after the release.

Would love to hear your thoughts.


[1]

KIP-95: Incremental Batch Processing for Kafka Streams
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-95%3A+Incremental+Batch+Processing+for+Kafka+Streams>

KIP-120: Cleanup Kafka Streams builder API
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-120%3A+Cleanup+Kafka+Streams+builder+API>

KIP-123: Allow per stream/table timestamp extractor
<https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=68714788>

KIP 130: Expose states of active tasks to KafkaStreams public API
<https://cwiki.apache.org/confluence/display/KAFKA/KIP+130%3A+Expose+states+of+active+tasks+to+KafkaStreams+public+API>

KIP-132: Augment KStream.print to allow extra parameters in the printed
string
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-132+-+Augment+KStream.print+to+allow+extra+parameters+in+the+printed+string>

KIP-138: Change punctuate semantics
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-138%3A+Change+punctuate+semantics>

KIP-147: Add missing type parameters to StateStoreSupplier factories and
KGroupedStream/Table methods
<https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69408481>

KIP-149: Enabling key access in ValueTransformer, ValueMapper, and
ValueJoiner
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-149%3A+Enabling+key+access+in+ValueTransformer%2C+ValueMapper%2C+and+ValueJoiner#KIP-149:EnablingkeyaccessinValueTransformer,ValueMapper,andValueJoiner-RejectedAlternatives>

KIP-150 - Kafka-Streams Cogroup
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup>

KIP 155 - Add range scan for windowed state stores
<https://cwiki.apache.org/confluence/display/KAFKA/KIP+155+-+Add+range+scan+for+windowed+state+stores>

KIP 156 Add option "dry run" to Streams application reset tool
<https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69410150>


-- 
-- Guozhang

Re: [DISCUSS] Modify / Remove "Unstable" annotations in Streams API

Posted by Guozhang Wang <wa...@gmail.com>.
If there is no objections on this, I will go ahead and make the
corresponding changes before the code freeze of 0.11.0.0.

Guozhang

On Wed, May 17, 2017 at 11:24 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Re store API: Maybe classes like `Stores` need to be modified, so I agree
> it's safer to maintain it as Evolving.
>
>
>
> Guozhang
>
>
> On Tue, May 16, 2017 at 3:57 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> About Windows:
>>
>> We got the request to add TimeWindows etc to public API anyway. I guess
>> it's helpful for people adding their own window aggregates using
>> low-level API.
>>
>> But as mentioned already, it was just a thought, and I am also not 100%
>> sure if we would gain much by changing the return type. Thus, if we
>> don't see any advantage, we can just keep it as is.
>>
>>
>> Same for stores: if you think the store API is stable, we don't need
>> @Envolving -- we just got couple of KIPs about store API lately, so I
>> wanted to double check only :)
>>
>>
>> -Matthias
>>
>> On 5/16/17 3:49 PM, Guozhang Wang wrote:
>> > Hello Matthias,
>> >
>> > Reply inlined.
>> >
>> > On Sat, May 13, 2017 at 1:40 PM, Matthias J. Sax <matthias@confluent.io
>> >
>> > wrote:
>> >
>> >> +1 for the overall proposal.
>> >>
>> >> Some comments:
>> >>
>> >> Currently, `Windowed#window()` returns type `Window` and I am wondering
>> >> if we should improve type safety here and return the actual window type
>> >> (ie, TimeWindow, SessionWindow, JoinWindow) etc.
>> >>
>> >> If this would be a useful improvement (frankly speaking, I am not 100%
>> >> sure if we need this), we should mark `Windowed` as @evolving, too.
>> >>
>> >>
>> > Hmm, I'm not sure, since TimeWindow / etc are internal classes. If we
>> want
>> > to expose them I need to expose all these internal classes as part of
>> the
>> > public APIs. But these internal classes do not expose any more functions
>> > for users to call either.
>> >
>> >
>> >>
>> >> I am also no 100% sure about package `state` (or did you mean "top
>> level
>> >> package "o.a.k.streams" as you mention "StreamsMetric" in the same
>> >> paragraph) -- are we sure it's stable enough to remove the annotation?
>> >> Or should we use @evolving here, too?
>> >>
>> >>
>> > Which class are you mostly concerning about?
>> >
>> >
>> >> With KIP-120 in the pipeline, we should also add @evolving to
>> >> KafkaStreams IMHO.
>> >>
>> >>
>> > Yeah I forgot to mention that for the top-level classes  should be
>> > @evolving, including StreamsMetrics (I was wrong in the previous email,
>> it
>> > is not in `o.a.k.streams.state`)
>> >
>> >
>> >>
>> >> -Matthias
>> >>
>> >> On 5/11/17 3:48 AM, Eno Thereska wrote:
>> >>> Sounds reasonable.
>> >>>
>> >>> Thanks,
>> >>> Eno
>> >>>> On May 11, 2017, at 7:39 AM, Ismael Juma <is...@juma.me.uk> wrote:
>> >>>>
>> >>>> Thanks for the proposal Guozhang. This sounds good to me.
>> >>>>
>> >>>> Ismael
>> >>>>
>> >>>> On Thu, May 11, 2017 at 6:02 AM, Guozhang Wang <wa...@gmail.com>
>> >> wrote:
>> >>>>
>> >>>>> Hello folks,
>> >>>>>
>> >>>>> As we are approaching the feature freeze deadline of 0.11.0.0, one
>> >> thing I
>> >>>>> realized is that currently the Streams public APIs are still marked
>> as
>> >>>>> "Unstable", which is to indicate that the API itself does not
>> provide
>> >>>>> guarantees about backward compatibility across releases. On the
>> other
>> >> hand,
>> >>>>> since Streams have now been widely adopted in production use cases
>> by
>> >> many
>> >>>>> organizations, we are in fact evolving its APIs in a much stricter
>> >> manner
>> >>>>> than "Unstable" allows us: for all the current Streams related KIP
>> >>>>> proposals under discussions right now [1], people have been working
>> >> hard to
>> >>>>> make sure none of them are going to break backward compatibility in
>> the
>> >>>>> coming releases. So I think it would be a good timing to change the
>> >> Streams
>> >>>>> API annotations.
>> >>>>>
>> >>>>> My proposal would be the following:
>> >>>>>
>> >>>>> 1. For "o.a.k.streams.errors" and "o.a.k.streams.state" packages:
>> >> remove
>> >>>>> the annotations except `StreamsMetrics`.
>> >>>>>
>> >>>>> 2. For "o.a.k.streams.kstream": remove the annotations except
>> >> "KStream",
>> >>>>> "KTable", "GroupedKStream", "GroupedKTable", "GlobalKTable" and
>> >>>>> "KStreamBuilder".
>> >>>>>
>> >>>>> 3. For all the other public classes, including
>> >> "o.a.k.streams.processor"
>> >>>>> and the above mentioned classes, change the annotation to
>> "Evolving",
>> >> which
>> >>>>> means "we might break compatibility at minor releases (i.e. 0.12.x,
>> >> 0.13.x,
>> >>>>> 1.0.x etc) only".
>> >>>>>
>> >>>>>
>> >>>>> The ultimate goal is to make sure we won't break anything going
>> >> forward,
>> >>>>> hence in the future we should remove all the annotations to make
>> that
>> >>>>> clear. The above changes in 0.11.0.0 is to give us some "buffer
>> time"
>> >> in
>> >>>>> case there are some major API change proposals after the release.
>> >>>>>
>> >>>>> Would love to hear your thoughts.
>> >>>>>
>> >>>>>
>> >>>>> [1]
>> >>>>>
>> >>>>> KIP-95: Incremental Batch Processing for Kafka Streams
>> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>> 95%3A+Incremental+Batch+Processing+for+Kafka+Streams>
>> >>>>>
>> >>>>> KIP-120: Cleanup Kafka Streams builder API
>> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>> 120%3A+Cleanup+Kafka+Streams+builder+API>
>> >>>>>
>> >>>>> KIP-123: Allow per stream/table timestamp extractor
>> >>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
>> >> action?pageId=68714788
>> >>>>>>
>> >>>>>
>> >>>>> KIP 130: Expose states of active tasks to KafkaStreams public API
>> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>> >>>>> 130%3A+Expose+states+of+active+tasks+to+KafkaStreams+public+API>
>> >>>>>
>> >>>>> KIP-132: Augment KStream.print to allow extra parameters in the
>> printed
>> >>>>> string
>> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>> 132+-+Augment+KStream.print+to+allow+extra+parameters+in+
>> >>>>> the+printed+string>
>> >>>>>
>> >>>>> KIP-138: Change punctuate semantics
>> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>> 138%3A+Change+punctuate+semantics>
>> >>>>>
>> >>>>> KIP-147: Add missing type parameters to StateStoreSupplier factories
>> >> and
>> >>>>> KGroupedStream/Table methods
>> >>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
>> >> action?pageId=69408481
>> >>>>>>
>> >>>>>
>> >>>>> KIP-149: Enabling key access in ValueTransformer, ValueMapper, and
>> >>>>> ValueJoiner
>> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>> 149%3A+Enabling+key+access+in+ValueTransformer%2C+ValueMappe
>> r%2C+and+
>> >>>>> ValueJoiner#KIP-149:EnablingkeyaccessinValueTransformer,
>> ValueMapper,
>> >>>>> andValueJoiner-RejectedAlternatives>
>> >>>>>
>> >>>>> KIP-150 - Kafka-Streams Cogroup
>> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> >>>>> 150+-+Kafka-Streams+Cogroup>
>> >>>>>
>> >>>>> KIP 155 - Add range scan for windowed state stores
>> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>> >>>>> 155+-+Add+range+scan+for+windowed+state+stores>
>> >>>>>
>> >>>>> KIP 156 Add option "dry run" to Streams application reset tool
>> >>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
>> >> action?pageId=69410150
>> >>>>>>
>> >>>>>
>> >>>>>
>> >>>>> --
>> >>>>> -- Guozhang
>> >>>>>
>> >>>
>> >>
>> >>
>> >
>> >
>>
>>
>
>
> --
> -- Guozhang
>



-- 
-- Guozhang

Re: [DISCUSS] Modify / Remove "Unstable" annotations in Streams API

Posted by Guozhang Wang <wa...@gmail.com>.
Re store API: Maybe classes like `Stores` need to be modified, so I agree
it's safer to maintain it as Evolving.



Guozhang


On Tue, May 16, 2017 at 3:57 PM, Matthias J. Sax <ma...@confluent.io>
wrote:

> About Windows:
>
> We got the request to add TimeWindows etc to public API anyway. I guess
> it's helpful for people adding their own window aggregates using
> low-level API.
>
> But as mentioned already, it was just a thought, and I am also not 100%
> sure if we would gain much by changing the return type. Thus, if we
> don't see any advantage, we can just keep it as is.
>
>
> Same for stores: if you think the store API is stable, we don't need
> @Envolving -- we just got couple of KIPs about store API lately, so I
> wanted to double check only :)
>
>
> -Matthias
>
> On 5/16/17 3:49 PM, Guozhang Wang wrote:
> > Hello Matthias,
> >
> > Reply inlined.
> >
> > On Sat, May 13, 2017 at 1:40 PM, Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> +1 for the overall proposal.
> >>
> >> Some comments:
> >>
> >> Currently, `Windowed#window()` returns type `Window` and I am wondering
> >> if we should improve type safety here and return the actual window type
> >> (ie, TimeWindow, SessionWindow, JoinWindow) etc.
> >>
> >> If this would be a useful improvement (frankly speaking, I am not 100%
> >> sure if we need this), we should mark `Windowed` as @evolving, too.
> >>
> >>
> > Hmm, I'm not sure, since TimeWindow / etc are internal classes. If we
> want
> > to expose them I need to expose all these internal classes as part of the
> > public APIs. But these internal classes do not expose any more functions
> > for users to call either.
> >
> >
> >>
> >> I am also no 100% sure about package `state` (or did you mean "top level
> >> package "o.a.k.streams" as you mention "StreamsMetric" in the same
> >> paragraph) -- are we sure it's stable enough to remove the annotation?
> >> Or should we use @evolving here, too?
> >>
> >>
> > Which class are you mostly concerning about?
> >
> >
> >> With KIP-120 in the pipeline, we should also add @evolving to
> >> KafkaStreams IMHO.
> >>
> >>
> > Yeah I forgot to mention that for the top-level classes  should be
> > @evolving, including StreamsMetrics (I was wrong in the previous email,
> it
> > is not in `o.a.k.streams.state`)
> >
> >
> >>
> >> -Matthias
> >>
> >> On 5/11/17 3:48 AM, Eno Thereska wrote:
> >>> Sounds reasonable.
> >>>
> >>> Thanks,
> >>> Eno
> >>>> On May 11, 2017, at 7:39 AM, Ismael Juma <is...@juma.me.uk> wrote:
> >>>>
> >>>> Thanks for the proposal Guozhang. This sounds good to me.
> >>>>
> >>>> Ismael
> >>>>
> >>>> On Thu, May 11, 2017 at 6:02 AM, Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>>
> >>>>> Hello folks,
> >>>>>
> >>>>> As we are approaching the feature freeze deadline of 0.11.0.0, one
> >> thing I
> >>>>> realized is that currently the Streams public APIs are still marked
> as
> >>>>> "Unstable", which is to indicate that the API itself does not provide
> >>>>> guarantees about backward compatibility across releases. On the other
> >> hand,
> >>>>> since Streams have now been widely adopted in production use cases by
> >> many
> >>>>> organizations, we are in fact evolving its APIs in a much stricter
> >> manner
> >>>>> than "Unstable" allows us: for all the current Streams related KIP
> >>>>> proposals under discussions right now [1], people have been working
> >> hard to
> >>>>> make sure none of them are going to break backward compatibility in
> the
> >>>>> coming releases. So I think it would be a good timing to change the
> >> Streams
> >>>>> API annotations.
> >>>>>
> >>>>> My proposal would be the following:
> >>>>>
> >>>>> 1. For "o.a.k.streams.errors" and "o.a.k.streams.state" packages:
> >> remove
> >>>>> the annotations except `StreamsMetrics`.
> >>>>>
> >>>>> 2. For "o.a.k.streams.kstream": remove the annotations except
> >> "KStream",
> >>>>> "KTable", "GroupedKStream", "GroupedKTable", "GlobalKTable" and
> >>>>> "KStreamBuilder".
> >>>>>
> >>>>> 3. For all the other public classes, including
> >> "o.a.k.streams.processor"
> >>>>> and the above mentioned classes, change the annotation to "Evolving",
> >> which
> >>>>> means "we might break compatibility at minor releases (i.e. 0.12.x,
> >> 0.13.x,
> >>>>> 1.0.x etc) only".
> >>>>>
> >>>>>
> >>>>> The ultimate goal is to make sure we won't break anything going
> >> forward,
> >>>>> hence in the future we should remove all the annotations to make that
> >>>>> clear. The above changes in 0.11.0.0 is to give us some "buffer time"
> >> in
> >>>>> case there are some major API change proposals after the release.
> >>>>>
> >>>>> Would love to hear your thoughts.
> >>>>>
> >>>>>
> >>>>> [1]
> >>>>>
> >>>>> KIP-95: Incremental Batch Processing for Kafka Streams
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>> 95%3A+Incremental+Batch+Processing+for+Kafka+Streams>
> >>>>>
> >>>>> KIP-120: Cleanup Kafka Streams builder API
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>> 120%3A+Cleanup+Kafka+Streams+builder+API>
> >>>>>
> >>>>> KIP-123: Allow per stream/table timestamp extractor
> >>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
> >> action?pageId=68714788
> >>>>>>
> >>>>>
> >>>>> KIP 130: Expose states of active tasks to KafkaStreams public API
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
> >>>>> 130%3A+Expose+states+of+active+tasks+to+KafkaStreams+public+API>
> >>>>>
> >>>>> KIP-132: Augment KStream.print to allow extra parameters in the
> printed
> >>>>> string
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>> 132+-+Augment+KStream.print+to+allow+extra+parameters+in+
> >>>>> the+printed+string>
> >>>>>
> >>>>> KIP-138: Change punctuate semantics
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>> 138%3A+Change+punctuate+semantics>
> >>>>>
> >>>>> KIP-147: Add missing type parameters to StateStoreSupplier factories
> >> and
> >>>>> KGroupedStream/Table methods
> >>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
> >> action?pageId=69408481
> >>>>>>
> >>>>>
> >>>>> KIP-149: Enabling key access in ValueTransformer, ValueMapper, and
> >>>>> ValueJoiner
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>> 149%3A+Enabling+key+access+in+ValueTransformer%2C+
> ValueMapper%2C+and+
> >>>>> ValueJoiner#KIP-149:EnablingkeyaccessinValueTransformer,ValueMapper,
> >>>>> andValueJoiner-RejectedAlternatives>
> >>>>>
> >>>>> KIP-150 - Kafka-Streams Cogroup
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>>>> 150+-+Kafka-Streams+Cogroup>
> >>>>>
> >>>>> KIP 155 - Add range scan for windowed state stores
> >>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
> >>>>> 155+-+Add+range+scan+for+windowed+state+stores>
> >>>>>
> >>>>> KIP 156 Add option "dry run" to Streams application reset tool
> >>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
> >> action?pageId=69410150
> >>>>>>
> >>>>>
> >>>>>
> >>>>> --
> >>>>> -- Guozhang
> >>>>>
> >>>
> >>
> >>
> >
> >
>
>


-- 
-- Guozhang

Re: [DISCUSS] Modify / Remove "Unstable" annotations in Streams API

Posted by "Matthias J. Sax" <ma...@confluent.io>.
About Windows:

We got the request to add TimeWindows etc to public API anyway. I guess
it's helpful for people adding their own window aggregates using
low-level API.

But as mentioned already, it was just a thought, and I am also not 100%
sure if we would gain much by changing the return type. Thus, if we
don't see any advantage, we can just keep it as is.


Same for stores: if you think the store API is stable, we don't need
@Envolving -- we just got couple of KIPs about store API lately, so I
wanted to double check only :)


-Matthias

On 5/16/17 3:49 PM, Guozhang Wang wrote:
> Hello Matthias,
> 
> Reply inlined.
> 
> On Sat, May 13, 2017 at 1:40 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> +1 for the overall proposal.
>>
>> Some comments:
>>
>> Currently, `Windowed#window()` returns type `Window` and I am wondering
>> if we should improve type safety here and return the actual window type
>> (ie, TimeWindow, SessionWindow, JoinWindow) etc.
>>
>> If this would be a useful improvement (frankly speaking, I am not 100%
>> sure if we need this), we should mark `Windowed` as @evolving, too.
>>
>>
> Hmm, I'm not sure, since TimeWindow / etc are internal classes. If we want
> to expose them I need to expose all these internal classes as part of the
> public APIs. But these internal classes do not expose any more functions
> for users to call either.
> 
> 
>>
>> I am also no 100% sure about package `state` (or did you mean "top level
>> package "o.a.k.streams" as you mention "StreamsMetric" in the same
>> paragraph) -- are we sure it's stable enough to remove the annotation?
>> Or should we use @evolving here, too?
>>
>>
> Which class are you mostly concerning about?
> 
> 
>> With KIP-120 in the pipeline, we should also add @evolving to
>> KafkaStreams IMHO.
>>
>>
> Yeah I forgot to mention that for the top-level classes  should be
> @evolving, including StreamsMetrics (I was wrong in the previous email, it
> is not in `o.a.k.streams.state`)
> 
> 
>>
>> -Matthias
>>
>> On 5/11/17 3:48 AM, Eno Thereska wrote:
>>> Sounds reasonable.
>>>
>>> Thanks,
>>> Eno
>>>> On May 11, 2017, at 7:39 AM, Ismael Juma <is...@juma.me.uk> wrote:
>>>>
>>>> Thanks for the proposal Guozhang. This sounds good to me.
>>>>
>>>> Ismael
>>>>
>>>> On Thu, May 11, 2017 at 6:02 AM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>>>>
>>>>> Hello folks,
>>>>>
>>>>> As we are approaching the feature freeze deadline of 0.11.0.0, one
>> thing I
>>>>> realized is that currently the Streams public APIs are still marked as
>>>>> "Unstable", which is to indicate that the API itself does not provide
>>>>> guarantees about backward compatibility across releases. On the other
>> hand,
>>>>> since Streams have now been widely adopted in production use cases by
>> many
>>>>> organizations, we are in fact evolving its APIs in a much stricter
>> manner
>>>>> than "Unstable" allows us: for all the current Streams related KIP
>>>>> proposals under discussions right now [1], people have been working
>> hard to
>>>>> make sure none of them are going to break backward compatibility in the
>>>>> coming releases. So I think it would be a good timing to change the
>> Streams
>>>>> API annotations.
>>>>>
>>>>> My proposal would be the following:
>>>>>
>>>>> 1. For "o.a.k.streams.errors" and "o.a.k.streams.state" packages:
>> remove
>>>>> the annotations except `StreamsMetrics`.
>>>>>
>>>>> 2. For "o.a.k.streams.kstream": remove the annotations except
>> "KStream",
>>>>> "KTable", "GroupedKStream", "GroupedKTable", "GlobalKTable" and
>>>>> "KStreamBuilder".
>>>>>
>>>>> 3. For all the other public classes, including
>> "o.a.k.streams.processor"
>>>>> and the above mentioned classes, change the annotation to "Evolving",
>> which
>>>>> means "we might break compatibility at minor releases (i.e. 0.12.x,
>> 0.13.x,
>>>>> 1.0.x etc) only".
>>>>>
>>>>>
>>>>> The ultimate goal is to make sure we won't break anything going
>> forward,
>>>>> hence in the future we should remove all the annotations to make that
>>>>> clear. The above changes in 0.11.0.0 is to give us some "buffer time"
>> in
>>>>> case there are some major API change proposals after the release.
>>>>>
>>>>> Would love to hear your thoughts.
>>>>>
>>>>>
>>>>> [1]
>>>>>
>>>>> KIP-95: Incremental Batch Processing for Kafka Streams
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 95%3A+Incremental+Batch+Processing+for+Kafka+Streams>
>>>>>
>>>>> KIP-120: Cleanup Kafka Streams builder API
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 120%3A+Cleanup+Kafka+Streams+builder+API>
>>>>>
>>>>> KIP-123: Allow per stream/table timestamp extractor
>>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
>> action?pageId=68714788
>>>>>>
>>>>>
>>>>> KIP 130: Expose states of active tasks to KafkaStreams public API
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>>>>> 130%3A+Expose+states+of+active+tasks+to+KafkaStreams+public+API>
>>>>>
>>>>> KIP-132: Augment KStream.print to allow extra parameters in the printed
>>>>> string
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 132+-+Augment+KStream.print+to+allow+extra+parameters+in+
>>>>> the+printed+string>
>>>>>
>>>>> KIP-138: Change punctuate semantics
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 138%3A+Change+punctuate+semantics>
>>>>>
>>>>> KIP-147: Add missing type parameters to StateStoreSupplier factories
>> and
>>>>> KGroupedStream/Table methods
>>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
>> action?pageId=69408481
>>>>>>
>>>>>
>>>>> KIP-149: Enabling key access in ValueTransformer, ValueMapper, and
>>>>> ValueJoiner
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 149%3A+Enabling+key+access+in+ValueTransformer%2C+ValueMapper%2C+and+
>>>>> ValueJoiner#KIP-149:EnablingkeyaccessinValueTransformer,ValueMapper,
>>>>> andValueJoiner-RejectedAlternatives>
>>>>>
>>>>> KIP-150 - Kafka-Streams Cogroup
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 150+-+Kafka-Streams+Cogroup>
>>>>>
>>>>> KIP 155 - Add range scan for windowed state stores
>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>>>>> 155+-+Add+range+scan+for+windowed+state+stores>
>>>>>
>>>>> KIP 156 Add option "dry run" to Streams application reset tool
>>>>> <https://cwiki.apache.org/confluence/pages/viewpage.
>> action?pageId=69410150
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>>>
>>>
>>
>>
> 
> 


Re: [DISCUSS] Modify / Remove "Unstable" annotations in Streams API

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Matthias,

Reply inlined.

On Sat, May 13, 2017 at 1:40 PM, Matthias J. Sax <ma...@confluent.io>
wrote:

> +1 for the overall proposal.
>
> Some comments:
>
> Currently, `Windowed#window()` returns type `Window` and I am wondering
> if we should improve type safety here and return the actual window type
> (ie, TimeWindow, SessionWindow, JoinWindow) etc.
>
> If this would be a useful improvement (frankly speaking, I am not 100%
> sure if we need this), we should mark `Windowed` as @evolving, too.
>
>
Hmm, I'm not sure, since TimeWindow / etc are internal classes. If we want
to expose them I need to expose all these internal classes as part of the
public APIs. But these internal classes do not expose any more functions
for users to call either.


>
> I am also no 100% sure about package `state` (or did you mean "top level
> package "o.a.k.streams" as you mention "StreamsMetric" in the same
> paragraph) -- are we sure it's stable enough to remove the annotation?
> Or should we use @evolving here, too?
>
>
Which class are you mostly concerning about?


> With KIP-120 in the pipeline, we should also add @evolving to
> KafkaStreams IMHO.
>
>
Yeah I forgot to mention that for the top-level classes  should be
@evolving, including StreamsMetrics (I was wrong in the previous email, it
is not in `o.a.k.streams.state`)


>
> -Matthias
>
> On 5/11/17 3:48 AM, Eno Thereska wrote:
> > Sounds reasonable.
> >
> > Thanks,
> > Eno
> >> On May 11, 2017, at 7:39 AM, Ismael Juma <is...@juma.me.uk> wrote:
> >>
> >> Thanks for the proposal Guozhang. This sounds good to me.
> >>
> >> Ismael
> >>
> >> On Thu, May 11, 2017 at 6:02 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >>> Hello folks,
> >>>
> >>> As we are approaching the feature freeze deadline of 0.11.0.0, one
> thing I
> >>> realized is that currently the Streams public APIs are still marked as
> >>> "Unstable", which is to indicate that the API itself does not provide
> >>> guarantees about backward compatibility across releases. On the other
> hand,
> >>> since Streams have now been widely adopted in production use cases by
> many
> >>> organizations, we are in fact evolving its APIs in a much stricter
> manner
> >>> than "Unstable" allows us: for all the current Streams related KIP
> >>> proposals under discussions right now [1], people have been working
> hard to
> >>> make sure none of them are going to break backward compatibility in the
> >>> coming releases. So I think it would be a good timing to change the
> Streams
> >>> API annotations.
> >>>
> >>> My proposal would be the following:
> >>>
> >>> 1. For "o.a.k.streams.errors" and "o.a.k.streams.state" packages:
> remove
> >>> the annotations except `StreamsMetrics`.
> >>>
> >>> 2. For "o.a.k.streams.kstream": remove the annotations except
> "KStream",
> >>> "KTable", "GroupedKStream", "GroupedKTable", "GlobalKTable" and
> >>> "KStreamBuilder".
> >>>
> >>> 3. For all the other public classes, including
> "o.a.k.streams.processor"
> >>> and the above mentioned classes, change the annotation to "Evolving",
> which
> >>> means "we might break compatibility at minor releases (i.e. 0.12.x,
> 0.13.x,
> >>> 1.0.x etc) only".
> >>>
> >>>
> >>> The ultimate goal is to make sure we won't break anything going
> forward,
> >>> hence in the future we should remove all the annotations to make that
> >>> clear. The above changes in 0.11.0.0 is to give us some "buffer time"
> in
> >>> case there are some major API change proposals after the release.
> >>>
> >>> Would love to hear your thoughts.
> >>>
> >>>
> >>> [1]
> >>>
> >>> KIP-95: Incremental Batch Processing for Kafka Streams
> >>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 95%3A+Incremental+Batch+Processing+for+Kafka+Streams>
> >>>
> >>> KIP-120: Cleanup Kafka Streams builder API
> >>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 120%3A+Cleanup+Kafka+Streams+builder+API>
> >>>
> >>> KIP-123: Allow per stream/table timestamp extractor
> >>> <https://cwiki.apache.org/confluence/pages/viewpage.
> action?pageId=68714788
> >>>>
> >>>
> >>> KIP 130: Expose states of active tasks to KafkaStreams public API
> >>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
> >>> 130%3A+Expose+states+of+active+tasks+to+KafkaStreams+public+API>
> >>>
> >>> KIP-132: Augment KStream.print to allow extra parameters in the printed
> >>> string
> >>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 132+-+Augment+KStream.print+to+allow+extra+parameters+in+
> >>> the+printed+string>
> >>>
> >>> KIP-138: Change punctuate semantics
> >>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 138%3A+Change+punctuate+semantics>
> >>>
> >>> KIP-147: Add missing type parameters to StateStoreSupplier factories
> and
> >>> KGroupedStream/Table methods
> >>> <https://cwiki.apache.org/confluence/pages/viewpage.
> action?pageId=69408481
> >>>>
> >>>
> >>> KIP-149: Enabling key access in ValueTransformer, ValueMapper, and
> >>> ValueJoiner
> >>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 149%3A+Enabling+key+access+in+ValueTransformer%2C+ValueMapper%2C+and+
> >>> ValueJoiner#KIP-149:EnablingkeyaccessinValueTransformer,ValueMapper,
> >>> andValueJoiner-RejectedAlternatives>
> >>>
> >>> KIP-150 - Kafka-Streams Cogroup
> >>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> >>> 150+-+Kafka-Streams+Cogroup>
> >>>
> >>> KIP 155 - Add range scan for windowed state stores
> >>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
> >>> 155+-+Add+range+scan+for+windowed+state+stores>
> >>>
> >>> KIP 156 Add option "dry run" to Streams application reset tool
> >>> <https://cwiki.apache.org/confluence/pages/viewpage.
> action?pageId=69410150
> >>>>
> >>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >
>
>


-- 
-- Guozhang

Re: [DISCUSS] Modify / Remove "Unstable" annotations in Streams API

Posted by "Matthias J. Sax" <ma...@confluent.io>.
+1 for the overall proposal.

Some comments:

Currently, `Windowed#window()` returns type `Window` and I am wondering
if we should improve type safety here and return the actual window type
(ie, TimeWindow, SessionWindow, JoinWindow) etc.

If this would be a useful improvement (frankly speaking, I am not 100%
sure if we need this), we should mark `Windowed` as @evolving, too.


I am also no 100% sure about package `state` (or did you mean "top level
package "o.a.k.streams" as you mention "StreamsMetric" in the same
paragraph) -- are we sure it's stable enough to remove the annotation?
Or should we use @evolving here, too?

With KIP-120 in the pipeline, we should also add @evolving to
KafkaStreams IMHO.


-Matthias

On 5/11/17 3:48 AM, Eno Thereska wrote:
> Sounds reasonable.
> 
> Thanks,
> Eno
>> On May 11, 2017, at 7:39 AM, Ismael Juma <is...@juma.me.uk> wrote:
>>
>> Thanks for the proposal Guozhang. This sounds good to me.
>>
>> Ismael
>>
>> On Thu, May 11, 2017 at 6:02 AM, Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> Hello folks,
>>>
>>> As we are approaching the feature freeze deadline of 0.11.0.0, one thing I
>>> realized is that currently the Streams public APIs are still marked as
>>> "Unstable", which is to indicate that the API itself does not provide
>>> guarantees about backward compatibility across releases. On the other hand,
>>> since Streams have now been widely adopted in production use cases by many
>>> organizations, we are in fact evolving its APIs in a much stricter manner
>>> than "Unstable" allows us: for all the current Streams related KIP
>>> proposals under discussions right now [1], people have been working hard to
>>> make sure none of them are going to break backward compatibility in the
>>> coming releases. So I think it would be a good timing to change the Streams
>>> API annotations.
>>>
>>> My proposal would be the following:
>>>
>>> 1. For "o.a.k.streams.errors" and "o.a.k.streams.state" packages: remove
>>> the annotations except `StreamsMetrics`.
>>>
>>> 2. For "o.a.k.streams.kstream": remove the annotations except "KStream",
>>> "KTable", "GroupedKStream", "GroupedKTable", "GlobalKTable" and
>>> "KStreamBuilder".
>>>
>>> 3. For all the other public classes, including "o.a.k.streams.processor"
>>> and the above mentioned classes, change the annotation to "Evolving", which
>>> means "we might break compatibility at minor releases (i.e. 0.12.x, 0.13.x,
>>> 1.0.x etc) only".
>>>
>>>
>>> The ultimate goal is to make sure we won't break anything going forward,
>>> hence in the future we should remove all the annotations to make that
>>> clear. The above changes in 0.11.0.0 is to give us some "buffer time" in
>>> case there are some major API change proposals after the release.
>>>
>>> Would love to hear your thoughts.
>>>
>>>
>>> [1]
>>>
>>> KIP-95: Incremental Batch Processing for Kafka Streams
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 95%3A+Incremental+Batch+Processing+for+Kafka+Streams>
>>>
>>> KIP-120: Cleanup Kafka Streams builder API
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 120%3A+Cleanup+Kafka+Streams+builder+API>
>>>
>>> KIP-123: Allow per stream/table timestamp extractor
>>> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=68714788
>>>>
>>>
>>> KIP 130: Expose states of active tasks to KafkaStreams public API
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>>> 130%3A+Expose+states+of+active+tasks+to+KafkaStreams+public+API>
>>>
>>> KIP-132: Augment KStream.print to allow extra parameters in the printed
>>> string
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 132+-+Augment+KStream.print+to+allow+extra+parameters+in+
>>> the+printed+string>
>>>
>>> KIP-138: Change punctuate semantics
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 138%3A+Change+punctuate+semantics>
>>>
>>> KIP-147: Add missing type parameters to StateStoreSupplier factories and
>>> KGroupedStream/Table methods
>>> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69408481
>>>>
>>>
>>> KIP-149: Enabling key access in ValueTransformer, ValueMapper, and
>>> ValueJoiner
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 149%3A+Enabling+key+access+in+ValueTransformer%2C+ValueMapper%2C+and+
>>> ValueJoiner#KIP-149:EnablingkeyaccessinValueTransformer,ValueMapper,
>>> andValueJoiner-RejectedAlternatives>
>>>
>>> KIP-150 - Kafka-Streams Cogroup
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 150+-+Kafka-Streams+Cogroup>
>>>
>>> KIP 155 - Add range scan for windowed state stores
>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>>> 155+-+Add+range+scan+for+windowed+state+stores>
>>>
>>> KIP 156 Add option "dry run" to Streams application reset tool
>>> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69410150
>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
> 


Re: [DISCUSS] Modify / Remove "Unstable" annotations in Streams API

Posted by Eno Thereska <en...@gmail.com>.
Sounds reasonable.

Thanks,
Eno
> On May 11, 2017, at 7:39 AM, Ismael Juma <is...@juma.me.uk> wrote:
> 
> Thanks for the proposal Guozhang. This sounds good to me.
> 
> Ismael
> 
> On Thu, May 11, 2017 at 6:02 AM, Guozhang Wang <wa...@gmail.com> wrote:
> 
>> Hello folks,
>> 
>> As we are approaching the feature freeze deadline of 0.11.0.0, one thing I
>> realized is that currently the Streams public APIs are still marked as
>> "Unstable", which is to indicate that the API itself does not provide
>> guarantees about backward compatibility across releases. On the other hand,
>> since Streams have now been widely adopted in production use cases by many
>> organizations, we are in fact evolving its APIs in a much stricter manner
>> than "Unstable" allows us: for all the current Streams related KIP
>> proposals under discussions right now [1], people have been working hard to
>> make sure none of them are going to break backward compatibility in the
>> coming releases. So I think it would be a good timing to change the Streams
>> API annotations.
>> 
>> My proposal would be the following:
>> 
>> 1. For "o.a.k.streams.errors" and "o.a.k.streams.state" packages: remove
>> the annotations except `StreamsMetrics`.
>> 
>> 2. For "o.a.k.streams.kstream": remove the annotations except "KStream",
>> "KTable", "GroupedKStream", "GroupedKTable", "GlobalKTable" and
>> "KStreamBuilder".
>> 
>> 3. For all the other public classes, including "o.a.k.streams.processor"
>> and the above mentioned classes, change the annotation to "Evolving", which
>> means "we might break compatibility at minor releases (i.e. 0.12.x, 0.13.x,
>> 1.0.x etc) only".
>> 
>> 
>> The ultimate goal is to make sure we won't break anything going forward,
>> hence in the future we should remove all the annotations to make that
>> clear. The above changes in 0.11.0.0 is to give us some "buffer time" in
>> case there are some major API change proposals after the release.
>> 
>> Would love to hear your thoughts.
>> 
>> 
>> [1]
>> 
>> KIP-95: Incremental Batch Processing for Kafka Streams
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 95%3A+Incremental+Batch+Processing+for+Kafka+Streams>
>> 
>> KIP-120: Cleanup Kafka Streams builder API
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 120%3A+Cleanup+Kafka+Streams+builder+API>
>> 
>> KIP-123: Allow per stream/table timestamp extractor
>> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=68714788
>>> 
>> 
>> KIP 130: Expose states of active tasks to KafkaStreams public API
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>> 130%3A+Expose+states+of+active+tasks+to+KafkaStreams+public+API>
>> 
>> KIP-132: Augment KStream.print to allow extra parameters in the printed
>> string
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 132+-+Augment+KStream.print+to+allow+extra+parameters+in+
>> the+printed+string>
>> 
>> KIP-138: Change punctuate semantics
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 138%3A+Change+punctuate+semantics>
>> 
>> KIP-147: Add missing type parameters to StateStoreSupplier factories and
>> KGroupedStream/Table methods
>> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69408481
>>> 
>> 
>> KIP-149: Enabling key access in ValueTransformer, ValueMapper, and
>> ValueJoiner
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 149%3A+Enabling+key+access+in+ValueTransformer%2C+ValueMapper%2C+and+
>> ValueJoiner#KIP-149:EnablingkeyaccessinValueTransformer,ValueMapper,
>> andValueJoiner-RejectedAlternatives>
>> 
>> KIP-150 - Kafka-Streams Cogroup
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 150+-+Kafka-Streams+Cogroup>
>> 
>> KIP 155 - Add range scan for windowed state stores
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>> 155+-+Add+range+scan+for+windowed+state+stores>
>> 
>> KIP 156 Add option "dry run" to Streams application reset tool
>> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69410150
>>> 
>> 
>> 
>> --
>> -- Guozhang
>> 


Re: [DISCUSS] Modify / Remove "Unstable" annotations in Streams API

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks for the proposal Guozhang. This sounds good to me.

Ismael

On Thu, May 11, 2017 at 6:02 AM, Guozhang Wang <wa...@gmail.com> wrote:

> Hello folks,
>
> As we are approaching the feature freeze deadline of 0.11.0.0, one thing I
> realized is that currently the Streams public APIs are still marked as
> "Unstable", which is to indicate that the API itself does not provide
> guarantees about backward compatibility across releases. On the other hand,
> since Streams have now been widely adopted in production use cases by many
> organizations, we are in fact evolving its APIs in a much stricter manner
> than "Unstable" allows us: for all the current Streams related KIP
> proposals under discussions right now [1], people have been working hard to
> make sure none of them are going to break backward compatibility in the
> coming releases. So I think it would be a good timing to change the Streams
> API annotations.
>
> My proposal would be the following:
>
> 1. For "o.a.k.streams.errors" and "o.a.k.streams.state" packages: remove
> the annotations except `StreamsMetrics`.
>
> 2. For "o.a.k.streams.kstream": remove the annotations except "KStream",
> "KTable", "GroupedKStream", "GroupedKTable", "GlobalKTable" and
> "KStreamBuilder".
>
> 3. For all the other public classes, including "o.a.k.streams.processor"
> and the above mentioned classes, change the annotation to "Evolving", which
> means "we might break compatibility at minor releases (i.e. 0.12.x, 0.13.x,
> 1.0.x etc) only".
>
>
> The ultimate goal is to make sure we won't break anything going forward,
> hence in the future we should remove all the annotations to make that
> clear. The above changes in 0.11.0.0 is to give us some "buffer time" in
> case there are some major API change proposals after the release.
>
> Would love to hear your thoughts.
>
>
> [1]
>
> KIP-95: Incremental Batch Processing for Kafka Streams
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 95%3A+Incremental+Batch+Processing+for+Kafka+Streams>
>
> KIP-120: Cleanup Kafka Streams builder API
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 120%3A+Cleanup+Kafka+Streams+builder+API>
>
> KIP-123: Allow per stream/table timestamp extractor
> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=68714788
> >
>
> KIP 130: Expose states of active tasks to KafkaStreams public API
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
> 130%3A+Expose+states+of+active+tasks+to+KafkaStreams+public+API>
>
> KIP-132: Augment KStream.print to allow extra parameters in the printed
> string
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 132+-+Augment+KStream.print+to+allow+extra+parameters+in+
> the+printed+string>
>
> KIP-138: Change punctuate semantics
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 138%3A+Change+punctuate+semantics>
>
> KIP-147: Add missing type parameters to StateStoreSupplier factories and
> KGroupedStream/Table methods
> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69408481
> >
>
> KIP-149: Enabling key access in ValueTransformer, ValueMapper, and
> ValueJoiner
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 149%3A+Enabling+key+access+in+ValueTransformer%2C+ValueMapper%2C+and+
> ValueJoiner#KIP-149:EnablingkeyaccessinValueTransformer,ValueMapper,
> andValueJoiner-RejectedAlternatives>
>
> KIP-150 - Kafka-Streams Cogroup
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 150+-+Kafka-Streams+Cogroup>
>
> KIP 155 - Add range scan for windowed state stores
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP+
> 155+-+Add+range+scan+for+windowed+state+stores>
>
> KIP 156 Add option "dry run" to Streams application reset tool
> <https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69410150
> >
>
>
> --
> -- Guozhang
>