I think the ListState interface is pretty well suited for this job. It allows to add elements with low effort and can serve all elements of a list through an iterator. Depending on the implementation the elements could be deserialized as needed. If the user code needs a List with all elements, it won't make a big performance difference whether the state backend or the user code copies the elements into a List object.
2017-05-18 15:24 GMT+02:00 Radu Tudoran <radu.tudo...@huawei.com>: > Actually that was one option that I was considering. I am still a bit > fuzzy about the advantages and disadvantages of using one type of state > over another. I know that using ValueState would mean that when getting the > object value (i.e. a List in this case) the whole would be deserialized at > once. This is ok if I need to go anyway through all elements. However, I > understand that when I need to update the state the same will hold - the > whole list would be serialized and re-write instead of a single element. > Therefore if we want to get always the best performances it seemed to me > that it would be worth considering have many specialized types of states - > hence my proposal. > > > > -----Original Message----- > From: Kostas Kloudas [mailto:k.klou...@data-artisans.com] > Sent: Thursday, May 18, 2017 11:49 AM > To: dev@flink.apache.org > Subject: Re: ListState to List > > Hi Radu, > > Why not using a ValueState that inside stored the whole list. > Whenever you state#get() you get the whole list and you can sort it. > > Kostas > > > On May 18, 2017, at 3:31 AM, Radu Tudoran <radu.tudo...@huawei.com> > wrote: > > > > Hi Aljoscha, > > > > Thanks for the clarification. I understand that there might be > advantages in some cases not to have the List-like interface, while in > other scenarios (like the one I described there aren't). Considering this, > why not having 2 type of states: ListState and StreamInListState - users > would use the one it is more appropriate. What do you think? > > > > -----Original Message----- > > From: Aljoscha Krettek [mailto:aljos...@apache.org] > > Sent: Thursday, May 18, 2017 12:15 AM > > To: dev@flink.apache.org > > Subject: Re: ListState to List > > > > Hi, > > The interface is restrictive on purpose because depending on the state > backend it might not be possible to provide a List-like interface. There > might be state backends that stream in the list from somewhere else or > other restrictions. If we now allowed a more general interface here we > would possibly prevent optimisations in the future or make certain > implementations very hard to to efficiently. > > > > Best, > > Aljoscha > > > >> On 16. May 2017, at 21:56, Radu Tudoran <radu.tudo...@huawei.com> > wrote: > >> > >> Hi, > >> > >> I would like to work with ListState, more specifically I would need to > access the contents and sort them. For this I would need a collection type > (e.g., the List, Array...). > >> However, I see that if I have a variable of type <<ListState state=..>> > the only interfaces I have are: > >> state.get -> which returns an Iterable Or state.get.getIterator which > >> returns an Iterator > >> > >> Basically if I use any of these I need now to copy the contents in an > actual List of Array. Is there any way to avoid this? ..perhaps there is > an implicit type that I can convert to... > >> > > > >