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 <m...@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 <reckone...@gmail.com> > wrote: > > > Wow, this looks pretty concise. I really like it! > > > > On Mon, Jun 29, 2015 at 3:27 PM Gyula Fóra <gyf...@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 > > > > > >