You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Maximilian Michels <mx...@apache.org> on 2015/07/01 10:10:01 UTC

Re: Replacing Checkpointed interface with field annotations

Hi Gyula,

Looks like a neat feature you thought of; I like it. One problem I see is
that with all the big changes and discussions in streaming, I wonder if we
should get other things right first. Another problem could be the lack of
clarity of this implementation. State annotations can now be very easily
placed anywhere in a class with the @Annotation. The Checkpointed interface
we have now is more explicit. From an end user perspective it might make
sense to use annotations, from a developer perspective I'm not sure.

Cheers,
Max



On Tue, Jun 30, 2015 at 1:44 PM, Hermann Gábor <re...@gmail.com> wrote:

> Wow, this looks pretty concise. I really like it!
>
> On Mon, Jun 29, 2015 at 3:27 PM Gyula Fóra <gy...@apache.org> wrote:
>
> > Hey all!
> >
> > Just to add something new to the end of the discussion list. After some
> > discussion with Seif, and Paris, I have added a commit that replaces the
> > use of the Checkpointed interface with field annotations.
> >
> > This is probably the most lightweight state declaration so far and it
> will
> > probably work very well to replace the Checkpointed interface:
> >
> > public class StatefulMapper implements MapFunction<Integer,Integer> {
> >
> > @State
> > int counter;
> > @State
> > Serializable state;
> >
> > Object notState
> >
> > public Integer map(Integer input)[
> > counter++;
> > //update other state
> > /...
> > }
> > }
> >
> > What do you think?
> > You can check it out here
> > <https://github.com/gyfora/flink/commits/annotated_state>.
> >
> > Cheers,
> > Gyula
> >
>

Re: Replacing Checkpointed interface with field annotations

Posted by Paris Carbone <pa...@kth.se>.
+1 on offering both. 

This way we cover both simplicity and expressivity when needed. Annotations give a very clean and simple way for marking state imho.
Perhaps most casual users will find it much better to just tag the fields that they want to persist.

Paris

> On 01 Jul 2015, at 15:55, Stephan Ewen <se...@apache.org> wrote:
> 
> Actually, this is the first rework of the state interface. There is only
> one released version, yet.
> What we are doing here is trying to make sure that this first rework will
> most likely also be the last for the foreseeable future.
> 
> 
> From the use cases I can think of, we need at least two different state
> checkpointing methods:
> 
> 
> 1) The variant where state is abstracted as a key/value interface. This is
> the new partitionable state interface.
>    Backuped is exactly what you put into the state. No need to make the
> operator aware about when checkpoint happen.
>    Most simple applications should be able to work against this interface.
> 
> 
> 2) The variant where the user code gets a call onCheckpoint() (currently
> snapshotState()) and returns whatever it wants to be persisted. This is
> important if the streaming
>    flow interacts with outside systems and wants to "groupCommit" data on
> checkpoints.
> 
>    The crucial thing here is that the value-to-be persisted by Flink may
> be in some cases not the actual data - that one has been periodically
> inserted into the external system.
>    The checkpointed value is only a key, epoch counter, or transaction ID
> that allows you mark what has been inserted into the external system as
> part of that checkpoint.
> 
>    This call to "onCheckpoint()" is not best-effort, but crucial and needs
> to succeed if a checkpoint is to be successful. Best effort is only
> "notifyCompleteCheckpoint()".
>    And we could make this message "at-least-once", if that is needed for
> reliable interaction with the outside world.
> 
>    In the last Flink meetup in the Bay Area, we had quite a discussion
> with some people about how interface (2) is powerful when trying to get
> "exactly-once" with external systems.
> 
>    Also, with this interface, it is quite straightforward to make
> asynchronous snapshotting possible, and it can be extended to incremental
> snapshotting. It is not obvious to me
>    how the same should work on the annotation variant.
> 
> 
> Concerning the annotated state:
> 
> That is eye candy and nice. Would it hurt to have this and promote it as a
> "shortcut" to a state backup implementation using (2), where the
> "shapshotState" method would simply
> return the value of some fields?
> 
> I know we should not offer too many different ways of doing things, but if
> we promote (2) as "2-general" (interface) and "2-shortcut" (annotation), I
> see no problem.
> 
> 
> Greetings,
> Stephan
> 
> 
> 
> 
> On Wed, Jul 1, 2015 at 11:59 AM, Robert Metzger <rm...@apache.org> wrote:
> 
>> I agree, if we want to change the interface, now is the best time.
>> 
>> So you are suggesting to change the methods in the Checkpointed interface
>> from
>> 
>> T snapshotState(long checkpointId, long checkpointTimestamp) throws
>> Exception;
>> 
>> void restoreState(T state);
>> 
>> to
>> 
>> void onSnapshot(id, ts)
>> void onRestore(id, ts)
>> (+ user has to annotate checkpointed fields)
>> 
>> I would say that the current interface is more powerful than what you
>> are proposing (arguments will follow)
>> I don't think that there is an advantage in usability for the user
>> with the new methods (but that is a matter of taste ... )
>> 
>> I think that the current interface is more powerful because it allows
>> you to give the system a derived state to back up, instead of just the
>> value of a variable. You would need to always update the derived state
>> so that the system can back it up when it needs to.
>> With the method, you can do this set only on demand.
>> For the restore method, with the old interface, you can do sanity
>> checks on the state to restore (something the only user of these
>> interfaces (the kafka source) is actually doing). With your proposed
>> interface, I would need to validate data from a field.
>> The proposed restore method would also make it harder to restore from
>> a derived state.
>> 
>> 
>> On Wed, Jul 1, 2015 at 11:38 AM, Gyula Fóra <gy...@gmail.com> wrote:
>> 
>>> I understand your concerns Robert but I don't fully agree.
>>> 
>>> The Checkpointed interface works indeed but there are so many use cases
>>> that it is not suitable for in the long run, and also the whole interface
>>> is slightly awkward in my opinion when returning simple fields which are
>>> already serializable.
>>> 
>>> This motivated the introduction of the OperatorStateInterface which you
>> can
>>> call the first rework of the checkpointed interface, but I see that as
>> the
>>> first version which is actually capable of handling many issues that were
>>> obvious with the Checkpointed interfaces.
>>> 
>>> This is actually not only a rework of the interface but the rework of the
>>> state concept and runtime handling. This needs to be clean if we are
>> moving
>>> streaming out of beta, and should provide the needed funcionality. I
>> think
>>> we can afford to experiment around a little bit with these interfaces and
>>> see the implications for the applications that we can develop with them
>> as
>>> we think of statefulness as a major advantage of Flink streaming.
>>> 
>>> So actually I think this is the only time when we can afford rework these
>>> interfaces without big costs to make it work for the future.
>>> 
>>> 
>>> 
>>> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. júl. 1.,
>>> Sze,
>>> 11:25):
>>> 
>>>> Whats causing me the biggest headache here is that I don't see an end
>> on
>>>> all these "state interface" reworks.
>>>> I think this is now the third big change to the interface.
>>>> It is a horrible user experience to rework your old code with each new
>>>> Flink release.
>>>> 
>>>> I understand that there are always ways to improve interfaces, and I'm
>>> sure
>>>> Flink has many that we can improve.
>>>> But there are (in my opinion) more important things than reworking the
>>>> interfaces every second week ... for example that the functionality
>> they
>>>> are providing is actually working and well tested.
>>>> 
>>>> 
>>>> 
>>>> On Wed, Jul 1, 2015 at 11:15 AM, Ufuk Celebi <uc...@apache.org> wrote:
>>>> 
>>>>> 
>>>>> On 01 Jul 2015, at 10:57, Gyula Fóra <gy...@gmail.com> wrote:
>>>>> 
>>>>>> Hey,
>>>>>> 
>>>>>> Thanks for the feedback guys:
>>>>>> 
>>>>>> @Max: You are right, this is not top priority to changes, I was
>> just
>>>>>> mocking up some alternatives to try to make the state usage even
>>>> simpler
>>>>> so
>>>>>> that the user can keep his current implementations and just add 1-2
>>>>>> annotations.
>>>>> 
>>>>> I agree. It's good to cover the "basic" case with a simple solution.
>>> :-)
>>>>> 
>>>>>> @Stephan, Robert: You are right that the checkpointed interface has
>>>> some
>>>>>> advantages from that point of view. Maybe a way to go would be to
>>>>> separate
>>>>>> this signaling functionality (when the checkpoint is taken and
>> maybe
>>>> also
>>>>>> the commits) from the snapshotting itself. One advantage I see
>> there
>>> is
>>>>>> that we would not need to have 3 different interfaces doing pretty
>>> much
>>>>> the
>>>>>> same thing (OperatorState - needed for partitioned state and
>>> different
>>>>>> backends/out-of-core, Checkpointed - needed for special actions
>> after
>>>>>> checkpoints, Annotations - checkpointing simple fields natively).
>>>>> 
>>>>> I also agree with Stephan and Robert that there are other use cases,
>>>> which
>>>>> require the interfaces. I cannot judge your proposal at this point
>>>> though.
>>>>> I'm eager to hear what the others say who worked on this.
>>>>> 
>>>>> – Ufuk
>>>> 
>>> 
>> 


Re: Replacing Checkpointed interface with field annotations

Posted by Stephan Ewen <se...@apache.org>.
Actually, this is the first rework of the state interface. There is only
one released version, yet.
What we are doing here is trying to make sure that this first rework will
most likely also be the last for the foreseeable future.


>From the use cases I can think of, we need at least two different state
checkpointing methods:


1) The variant where state is abstracted as a key/value interface. This is
the new partitionable state interface.
    Backuped is exactly what you put into the state. No need to make the
operator aware about when checkpoint happen.
    Most simple applications should be able to work against this interface.


2) The variant where the user code gets a call onCheckpoint() (currently
snapshotState()) and returns whatever it wants to be persisted. This is
important if the streaming
    flow interacts with outside systems and wants to "groupCommit" data on
checkpoints.

    The crucial thing here is that the value-to-be persisted by Flink may
be in some cases not the actual data - that one has been periodically
inserted into the external system.
    The checkpointed value is only a key, epoch counter, or transaction ID
that allows you mark what has been inserted into the external system as
part of that checkpoint.

    This call to "onCheckpoint()" is not best-effort, but crucial and needs
to succeed if a checkpoint is to be successful. Best effort is only
"notifyCompleteCheckpoint()".
    And we could make this message "at-least-once", if that is needed for
reliable interaction with the outside world.

    In the last Flink meetup in the Bay Area, we had quite a discussion
with some people about how interface (2) is powerful when trying to get
"exactly-once" with external systems.

    Also, with this interface, it is quite straightforward to make
asynchronous snapshotting possible, and it can be extended to incremental
snapshotting. It is not obvious to me
    how the same should work on the annotation variant.


Concerning the annotated state:

That is eye candy and nice. Would it hurt to have this and promote it as a
"shortcut" to a state backup implementation using (2), where the
"shapshotState" method would simply
return the value of some fields?

I know we should not offer too many different ways of doing things, but if
we promote (2) as "2-general" (interface) and "2-shortcut" (annotation), I
see no problem.


Greetings,
Stephan




On Wed, Jul 1, 2015 at 11:59 AM, Robert Metzger <rm...@apache.org> wrote:

> I agree, if we want to change the interface, now is the best time.
>
> So you are suggesting to change the methods in the Checkpointed interface
> from
>
> T snapshotState(long checkpointId, long checkpointTimestamp) throws
> Exception;
>
> void restoreState(T state);
>
> to
>
> void onSnapshot(id, ts)
> void onRestore(id, ts)
> (+ user has to annotate checkpointed fields)
>
> I would say that the current interface is more powerful than what you
> are proposing (arguments will follow)
> I don't think that there is an advantage in usability for the user
> with the new methods (but that is a matter of taste ... )
>
> I think that the current interface is more powerful because it allows
> you to give the system a derived state to back up, instead of just the
> value of a variable. You would need to always update the derived state
> so that the system can back it up when it needs to.
> With the method, you can do this set only on demand.
> For the restore method, with the old interface, you can do sanity
> checks on the state to restore (something the only user of these
> interfaces (the kafka source) is actually doing). With your proposed
> interface, I would need to validate data from a field.
> The proposed restore method would also make it harder to restore from
> a derived state.
>
>
> On Wed, Jul 1, 2015 at 11:38 AM, Gyula Fóra <gy...@gmail.com> wrote:
>
> > I understand your concerns Robert but I don't fully agree.
> >
> > The Checkpointed interface works indeed but there are so many use cases
> > that it is not suitable for in the long run, and also the whole interface
> > is slightly awkward in my opinion when returning simple fields which are
> > already serializable.
> >
> > This motivated the introduction of the OperatorStateInterface which you
> can
> > call the first rework of the checkpointed interface, but I see that as
> the
> > first version which is actually capable of handling many issues that were
> > obvious with the Checkpointed interfaces.
> >
> > This is actually not only a rework of the interface but the rework of the
> > state concept and runtime handling. This needs to be clean if we are
> moving
> > streaming out of beta, and should provide the needed funcionality. I
> think
> > we can afford to experiment around a little bit with these interfaces and
> > see the implications for the applications that we can develop with them
> as
> > we think of statefulness as a major advantage of Flink streaming.
> >
> > So actually I think this is the only time when we can afford rework these
> > interfaces without big costs to make it work for the future.
> >
> >
> >
> > Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. júl. 1.,
> > Sze,
> > 11:25):
> >
> > > Whats causing me the biggest headache here is that I don't see an end
> on
> > > all these "state interface" reworks.
> > > I think this is now the third big change to the interface.
> > > It is a horrible user experience to rework your old code with each new
> > > Flink release.
> > >
> > > I understand that there are always ways to improve interfaces, and I'm
> > sure
> > > Flink has many that we can improve.
> > > But there are (in my opinion) more important things than reworking the
> > > interfaces every second week ... for example that the functionality
> they
> > > are providing is actually working and well tested.
> > >
> > >
> > >
> > > On Wed, Jul 1, 2015 at 11:15 AM, Ufuk Celebi <uc...@apache.org> wrote:
> > >
> > > >
> > > > On 01 Jul 2015, at 10:57, Gyula Fóra <gy...@gmail.com> wrote:
> > > >
> > > > > Hey,
> > > > >
> > > > > Thanks for the feedback guys:
> > > > >
> > > > > @Max: You are right, this is not top priority to changes, I was
> just
> > > > > mocking up some alternatives to try to make the state usage even
> > > simpler
> > > > so
> > > > > that the user can keep his current implementations and just add 1-2
> > > > > annotations.
> > > >
> > > > I agree. It's good to cover the "basic" case with a simple solution.
> > :-)
> > > >
> > > > > @Stephan, Robert: You are right that the checkpointed interface has
> > > some
> > > > > advantages from that point of view. Maybe a way to go would be to
> > > > separate
> > > > > this signaling functionality (when the checkpoint is taken and
> maybe
> > > also
> > > > > the commits) from the snapshotting itself. One advantage I see
> there
> > is
> > > > > that we would not need to have 3 different interfaces doing pretty
> > much
> > > > the
> > > > > same thing (OperatorState - needed for partitioned state and
> > different
> > > > > backends/out-of-core, Checkpointed - needed for special actions
> after
> > > > > checkpoints, Annotations - checkpointing simple fields natively).
> > > >
> > > > I also agree with Stephan and Robert that there are other use cases,
> > > which
> > > > require the interfaces. I cannot judge your proposal at this point
> > > though.
> > > > I'm eager to hear what the others say who worked on this.
> > > >
> > > > – Ufuk
> > >
> >
>

Re: Replacing Checkpointed interface with field annotations

Posted by Robert Metzger <rm...@apache.org>.
I agree, if we want to change the interface, now is the best time.

So you are suggesting to change the methods in the Checkpointed interface
from

T snapshotState(long checkpointId, long checkpointTimestamp) throws Exception;

void restoreState(T state);

to

void onSnapshot(id, ts)
void onRestore(id, ts)
(+ user has to annotate checkpointed fields)

I would say that the current interface is more powerful than what you
are proposing (arguments will follow)
I don't think that there is an advantage in usability for the user
with the new methods (but that is a matter of taste ... )

I think that the current interface is more powerful because it allows
you to give the system a derived state to back up, instead of just the
value of a variable. You would need to always update the derived state
so that the system can back it up when it needs to.
With the method, you can do this set only on demand.
For the restore method, with the old interface, you can do sanity
checks on the state to restore (something the only user of these
interfaces (the kafka source) is actually doing). With your proposed
interface, I would need to validate data from a field.
The proposed restore method would also make it harder to restore from
a derived state.


On Wed, Jul 1, 2015 at 11:38 AM, Gyula Fóra <gy...@gmail.com> wrote:

> I understand your concerns Robert but I don't fully agree.
>
> The Checkpointed interface works indeed but there are so many use cases
> that it is not suitable for in the long run, and also the whole interface
> is slightly awkward in my opinion when returning simple fields which are
> already serializable.
>
> This motivated the introduction of the OperatorStateInterface which you can
> call the first rework of the checkpointed interface, but I see that as the
> first version which is actually capable of handling many issues that were
> obvious with the Checkpointed interfaces.
>
> This is actually not only a rework of the interface but the rework of the
> state concept and runtime handling. This needs to be clean if we are moving
> streaming out of beta, and should provide the needed funcionality. I think
> we can afford to experiment around a little bit with these interfaces and
> see the implications for the applications that we can develop with them as
> we think of statefulness as a major advantage of Flink streaming.
>
> So actually I think this is the only time when we can afford rework these
> interfaces without big costs to make it work for the future.
>
>
>
> Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. júl. 1.,
> Sze,
> 11:25):
>
> > Whats causing me the biggest headache here is that I don't see an end on
> > all these "state interface" reworks.
> > I think this is now the third big change to the interface.
> > It is a horrible user experience to rework your old code with each new
> > Flink release.
> >
> > I understand that there are always ways to improve interfaces, and I'm
> sure
> > Flink has many that we can improve.
> > But there are (in my opinion) more important things than reworking the
> > interfaces every second week ... for example that the functionality they
> > are providing is actually working and well tested.
> >
> >
> >
> > On Wed, Jul 1, 2015 at 11:15 AM, Ufuk Celebi <uc...@apache.org> wrote:
> >
> > >
> > > On 01 Jul 2015, at 10:57, Gyula Fóra <gy...@gmail.com> wrote:
> > >
> > > > Hey,
> > > >
> > > > Thanks for the feedback guys:
> > > >
> > > > @Max: You are right, this is not top priority to changes, I was just
> > > > mocking up some alternatives to try to make the state usage even
> > simpler
> > > so
> > > > that the user can keep his current implementations and just add 1-2
> > > > annotations.
> > >
> > > I agree. It's good to cover the "basic" case with a simple solution.
> :-)
> > >
> > > > @Stephan, Robert: You are right that the checkpointed interface has
> > some
> > > > advantages from that point of view. Maybe a way to go would be to
> > > separate
> > > > this signaling functionality (when the checkpoint is taken and maybe
> > also
> > > > the commits) from the snapshotting itself. One advantage I see there
> is
> > > > that we would not need to have 3 different interfaces doing pretty
> much
> > > the
> > > > same thing (OperatorState - needed for partitioned state and
> different
> > > > backends/out-of-core, Checkpointed - needed for special actions after
> > > > checkpoints, Annotations - checkpointing simple fields natively).
> > >
> > > I also agree with Stephan and Robert that there are other use cases,
> > which
> > > require the interfaces. I cannot judge your proposal at this point
> > though.
> > > I'm eager to hear what the others say who worked on this.
> > >
> > > – Ufuk
> >
>

Re: Replacing Checkpointed interface with field annotations

Posted by Gyula Fóra <gy...@gmail.com>.
I understand your concerns Robert but I don't fully agree.

The Checkpointed interface works indeed but there are so many use cases
that it is not suitable for in the long run, and also the whole interface
is slightly awkward in my opinion when returning simple fields which are
already serializable.

This motivated the introduction of the OperatorStateInterface which you can
call the first rework of the checkpointed interface, but I see that as the
first version which is actually capable of handling many issues that were
obvious with the Checkpointed interfaces.

This is actually not only a rework of the interface but the rework of the
state concept and runtime handling. This needs to be clean if we are moving
streaming out of beta, and should provide the needed funcionality. I think
we can afford to experiment around a little bit with these interfaces and
see the implications for the applications that we can develop with them as
we think of statefulness as a major advantage of Flink streaming.

So actually I think this is the only time when we can afford rework these
interfaces without big costs to make it work for the future.



Robert Metzger <rm...@apache.org> ezt írta (időpont: 2015. júl. 1., Sze,
11:25):

> Whats causing me the biggest headache here is that I don't see an end on
> all these "state interface" reworks.
> I think this is now the third big change to the interface.
> It is a horrible user experience to rework your old code with each new
> Flink release.
>
> I understand that there are always ways to improve interfaces, and I'm sure
> Flink has many that we can improve.
> But there are (in my opinion) more important things than reworking the
> interfaces every second week ... for example that the functionality they
> are providing is actually working and well tested.
>
>
>
> On Wed, Jul 1, 2015 at 11:15 AM, Ufuk Celebi <uc...@apache.org> wrote:
>
> >
> > On 01 Jul 2015, at 10:57, Gyula Fóra <gy...@gmail.com> wrote:
> >
> > > Hey,
> > >
> > > Thanks for the feedback guys:
> > >
> > > @Max: You are right, this is not top priority to changes, I was just
> > > mocking up some alternatives to try to make the state usage even
> simpler
> > so
> > > that the user can keep his current implementations and just add 1-2
> > > annotations.
> >
> > I agree. It's good to cover the "basic" case with a simple solution. :-)
> >
> > > @Stephan, Robert: You are right that the checkpointed interface has
> some
> > > advantages from that point of view. Maybe a way to go would be to
> > separate
> > > this signaling functionality (when the checkpoint is taken and maybe
> also
> > > the commits) from the snapshotting itself. One advantage I see there is
> > > that we would not need to have 3 different interfaces doing pretty much
> > the
> > > same thing (OperatorState - needed for partitioned state and different
> > > backends/out-of-core, Checkpointed - needed for special actions after
> > > checkpoints, Annotations - checkpointing simple fields natively).
> >
> > I also agree with Stephan and Robert that there are other use cases,
> which
> > require the interfaces. I cannot judge your proposal at this point
> though.
> > I'm eager to hear what the others say who worked on this.
> >
> > – Ufuk
>

Re: Replacing Checkpointed interface with field annotations

Posted by Robert Metzger <rm...@apache.org>.
Whats causing me the biggest headache here is that I don't see an end on
all these "state interface" reworks.
I think this is now the third big change to the interface.
It is a horrible user experience to rework your old code with each new
Flink release.

I understand that there are always ways to improve interfaces, and I'm sure
Flink has many that we can improve.
But there are (in my opinion) more important things than reworking the
interfaces every second week ... for example that the functionality they
are providing is actually working and well tested.



On Wed, Jul 1, 2015 at 11:15 AM, Ufuk Celebi <uc...@apache.org> wrote:

>
> On 01 Jul 2015, at 10:57, Gyula Fóra <gy...@gmail.com> wrote:
>
> > Hey,
> >
> > Thanks for the feedback guys:
> >
> > @Max: You are right, this is not top priority to changes, I was just
> > mocking up some alternatives to try to make the state usage even simpler
> so
> > that the user can keep his current implementations and just add 1-2
> > annotations.
>
> I agree. It's good to cover the "basic" case with a simple solution. :-)
>
> > @Stephan, Robert: You are right that the checkpointed interface has some
> > advantages from that point of view. Maybe a way to go would be to
> separate
> > this signaling functionality (when the checkpoint is taken and maybe also
> > the commits) from the snapshotting itself. One advantage I see there is
> > that we would not need to have 3 different interfaces doing pretty much
> the
> > same thing (OperatorState - needed for partitioned state and different
> > backends/out-of-core, Checkpointed - needed for special actions after
> > checkpoints, Annotations - checkpointing simple fields natively).
>
> I also agree with Stephan and Robert that there are other use cases, which
> require the interfaces. I cannot judge your proposal at this point though.
> I'm eager to hear what the others say who worked on this.
>
> – Ufuk

Re: Replacing Checkpointed interface with field annotations

Posted by Ufuk Celebi <uc...@apache.org>.
On 01 Jul 2015, at 10:57, Gyula Fóra <gy...@gmail.com> wrote:

> Hey,
> 
> Thanks for the feedback guys:
> 
> @Max: You are right, this is not top priority to changes, I was just
> mocking up some alternatives to try to make the state usage even simpler so
> that the user can keep his current implementations and just add 1-2
> annotations.

I agree. It's good to cover the "basic" case with a simple solution. :-)

> @Stephan, Robert: You are right that the checkpointed interface has some
> advantages from that point of view. Maybe a way to go would be to separate
> this signaling functionality (when the checkpoint is taken and maybe also
> the commits) from the snapshotting itself. One advantage I see there is
> that we would not need to have 3 different interfaces doing pretty much the
> same thing (OperatorState - needed for partitioned state and different
> backends/out-of-core, Checkpointed - needed for special actions after
> checkpoints, Annotations - checkpointing simple fields natively).

I also agree with Stephan and Robert that there are other use cases, which require the interfaces. I cannot judge your proposal at this point though. I'm eager to hear what the others say who worked on this.

– Ufuk

Re: Replacing Checkpointed interface with field annotations

Posted by Gyula Fóra <gy...@gmail.com>.
Hey,

Thanks for the feedback guys:

@Max: You are right, this is not top priority to changes, I was just
mocking up some alternatives to try to make the state usage even simpler so
that the user can keep his current implementations and just add 1-2
annotations.

@Stephan, Robert: You are right that the checkpointed interface has some
advantages from that point of view. Maybe a way to go would be to separate
this signaling functionality (when the checkpoint is taken and maybe also
the commits) from the snapshotting itself. One advantage I see there is
that we would not need to have 3 different interfaces doing pretty much the
same thing (OperatorState - needed for partitioned state and different
backends/out-of-core, Checkpointed - needed for special actions after
checkpoints, Annotations - checkpointing simple fields natively).

What we could do is to modify the checkpointed interface, so it cannot
actually take a snapshot, but perform actions after the snapshot is taken,
and also on restore (and maybe include the commit functonality as well).

Checkpointed{
void onSnapshot(id, ts)
void onRestore(id, ts)
// void commit(id, ts)
}

Since all these special actions are best effort anyways I think this could
work. And now it is clear that OperatorStates and annotated fields would be
checkpointed, but we can still have actions on the checkpoints.

Would something like this make sense? I think it would be very good to
clarify and make the minimal necessary interfaces that don't necessarily
replicate functionality.

Cheers,
Gyula

Stephan Ewen <se...@apache.org> ezt írta (időpont: 2015. júl. 1., Sze,
10:22):

> +1 for adding the annotation, but not removing the interface
>
> Robert is right, the nice thing about the current interface is that you can
> use it to commit the state yourself to a database and simply return a key
> to where the state is stored. That is quite nice.
>
> On Wed, Jul 1, 2015 at 10:14 AM, Robert Metzger <rm...@apache.org>
> wrote:
>
> > I would certainly not replace the current Checkpointed interface by this
> > implementation.
> > The interface allows you to perform custom actions when creating a
> snapshot
> > or restoring state.
> >
> > We could add the annotation variant for simple cases like in your
> example.
> >
> >
> > On Wed, Jul 1, 2015 at 10:10 AM, Maximilian Michels <mx...@apache.org>
> > wrote:
> >
> > > Hi Gyula,
> > >
> > > Looks like a neat feature you thought of; I like it. One problem I see
> is
> > > that with all the big changes and discussions in streaming, I wonder if
> > we
> > > should get other things right first. Another problem could be the lack
> of
> > > clarity of this implementation. State annotations can now be very
> easily
> > > placed anywhere in a class with the @Annotation. The Checkpointed
> > interface
> > > we have now is more explicit. From an end user perspective it might
> make
> > > sense to use annotations, from a developer perspective I'm not sure.
> > >
> > > Cheers,
> > > Max
> > >
> > >
> > >
> > > On Tue, Jun 30, 2015 at 1:44 PM, Hermann Gábor <re...@gmail.com>
> > > wrote:
> > >
> > > > Wow, this looks pretty concise. I really like it!
> > > >
> > > > On Mon, Jun 29, 2015 at 3:27 PM Gyula Fóra <gy...@apache.org>
> wrote:
> > > >
> > > > > Hey all!
> > > > >
> > > > > Just to add something new to the end of the discussion list. After
> > some
> > > > > discussion with Seif, and Paris, I have added a commit that
> replaces
> > > the
> > > > > use of the Checkpointed interface with field annotations.
> > > > >
> > > > > This is probably the most lightweight state declaration so far and
> it
> > > > will
> > > > > probably work very well to replace the Checkpointed interface:
> > > > >
> > > > > public class StatefulMapper implements
> MapFunction<Integer,Integer> {
> > > > >
> > > > > @State
> > > > > int counter;
> > > > > @State
> > > > > Serializable state;
> > > > >
> > > > > Object notState
> > > > >
> > > > > public Integer map(Integer input)[
> > > > > counter++;
> > > > > //update other state
> > > > > /...
> > > > > }
> > > > > }
> > > > >
> > > > > What do you think?
> > > > > You can check it out here
> > > > > <https://github.com/gyfora/flink/commits/annotated_state>.
> > > > >
> > > > > Cheers,
> > > > > Gyula
> > > > >
> > > >
> > >
> >
>

Re: Replacing Checkpointed interface with field annotations

Posted by Stephan Ewen <se...@apache.org>.
+1 for adding the annotation, but not removing the interface

Robert is right, the nice thing about the current interface is that you can
use it to commit the state yourself to a database and simply return a key
to where the state is stored. That is quite nice.

On Wed, Jul 1, 2015 at 10:14 AM, Robert Metzger <rm...@apache.org> wrote:

> I would certainly not replace the current Checkpointed interface by this
> implementation.
> The interface allows you to perform custom actions when creating a snapshot
> or restoring state.
>
> We could add the annotation variant for simple cases like in your example.
>
>
> On Wed, Jul 1, 2015 at 10:10 AM, Maximilian Michels <mx...@apache.org>
> wrote:
>
> > Hi Gyula,
> >
> > Looks like a neat feature you thought of; I like it. One problem I see is
> > that with all the big changes and discussions in streaming, I wonder if
> we
> > should get other things right first. Another problem could be the lack of
> > clarity of this implementation. State annotations can now be very easily
> > placed anywhere in a class with the @Annotation. The Checkpointed
> interface
> > we have now is more explicit. From an end user perspective it might make
> > sense to use annotations, from a developer perspective I'm not sure.
> >
> > Cheers,
> > Max
> >
> >
> >
> > On Tue, Jun 30, 2015 at 1:44 PM, Hermann Gábor <re...@gmail.com>
> > wrote:
> >
> > > Wow, this looks pretty concise. I really like it!
> > >
> > > On Mon, Jun 29, 2015 at 3:27 PM Gyula Fóra <gy...@apache.org> wrote:
> > >
> > > > Hey all!
> > > >
> > > > Just to add something new to the end of the discussion list. After
> some
> > > > discussion with Seif, and Paris, I have added a commit that replaces
> > the
> > > > use of the Checkpointed interface with field annotations.
> > > >
> > > > This is probably the most lightweight state declaration so far and it
> > > will
> > > > probably work very well to replace the Checkpointed interface:
> > > >
> > > > public class StatefulMapper implements MapFunction<Integer,Integer> {
> > > >
> > > > @State
> > > > int counter;
> > > > @State
> > > > Serializable state;
> > > >
> > > > Object notState
> > > >
> > > > public Integer map(Integer input)[
> > > > counter++;
> > > > //update other state
> > > > /...
> > > > }
> > > > }
> > > >
> > > > What do you think?
> > > > You can check it out here
> > > > <https://github.com/gyfora/flink/commits/annotated_state>.
> > > >
> > > > Cheers,
> > > > Gyula
> > > >
> > >
> >
>

Re: Replacing Checkpointed interface with field annotations

Posted by Robert Metzger <rm...@apache.org>.
I would certainly not replace the current Checkpointed interface by this
implementation.
The interface allows you to perform custom actions when creating a snapshot
or restoring state.

We could add the annotation variant for simple cases like in your example.


On Wed, Jul 1, 2015 at 10:10 AM, Maximilian Michels <mx...@apache.org> wrote:

> Hi Gyula,
>
> Looks like a neat feature you thought of; I like it. One problem I see is
> that with all the big changes and discussions in streaming, I wonder if we
> should get other things right first. Another problem could be the lack of
> clarity of this implementation. State annotations can now be very easily
> placed anywhere in a class with the @Annotation. The Checkpointed interface
> we have now is more explicit. From an end user perspective it might make
> sense to use annotations, from a developer perspective I'm not sure.
>
> Cheers,
> Max
>
>
>
> On Tue, Jun 30, 2015 at 1:44 PM, Hermann Gábor <re...@gmail.com>
> wrote:
>
> > Wow, this looks pretty concise. I really like it!
> >
> > On Mon, Jun 29, 2015 at 3:27 PM Gyula Fóra <gy...@apache.org> wrote:
> >
> > > Hey all!
> > >
> > > Just to add something new to the end of the discussion list. After some
> > > discussion with Seif, and Paris, I have added a commit that replaces
> the
> > > use of the Checkpointed interface with field annotations.
> > >
> > > This is probably the most lightweight state declaration so far and it
> > will
> > > probably work very well to replace the Checkpointed interface:
> > >
> > > public class StatefulMapper implements MapFunction<Integer,Integer> {
> > >
> > > @State
> > > int counter;
> > > @State
> > > Serializable state;
> > >
> > > Object notState
> > >
> > > public Integer map(Integer input)[
> > > counter++;
> > > //update other state
> > > /...
> > > }
> > > }
> > >
> > > What do you think?
> > > You can check it out here
> > > <https://github.com/gyfora/flink/commits/annotated_state>.
> > >
> > > Cheers,
> > > Gyula
> > >
> >
>