Hi Vishal, As Fabian said, queryable state is just a feature that exposes the state kept within Flink, and it is not made to replace functionality that would otherwise be made by a sink. In the future the functionality will definitely evolve but for there are no discussions currently, for keeping the state of a job even after the job is done.
For being able to do so, with exactly once semantics and all the guarantees provided by Flink, I would recommend to use an external sink. Cheers, Kostas > On Mar 19, 2018, at 6:18 PM, Vishal Santoshi <vishal.santo...@gmail.com> > wrote: > > Thank you. These do look like show stoppers for us. But again thank you. > > On Mon, Mar 19, 2018 at 10:31 AM, Fabian Hueske <fhue...@gmail.com > <mailto:fhue...@gmail.com>> wrote: > AFAIK, there have been discussions to replicate state among TMs to speed up > recovery (and improve availability). > However, I'm not aware of plans to implement that. > > I don't think serving state while a job is down has been considered yet. > > 2018-03-19 15:17 GMT+01:00 Vishal Santoshi <vishal.santo...@gmail.com > <mailto:vishal.santo...@gmail.com>>: > Are there plans to address all or few of the above apart from the "JM LB not > possible" which seems understandable ? > > On Mon, Mar 19, 2018 at 9:58 AM, Fabian Hueske <fhue...@gmail.com > <mailto:fhue...@gmail.com>> wrote: > Queryable state is "just" an additional feature on regular keyed state. i.e., > the only difference is that you can read the state from an outside > application. > Besides that it behaves exactly like regular application state > > Queryable state is (at the moment) designed to be accessible if a job runs. > If the job fails (and recovers) or is manually taken down for maintenance, > the state cannot be queried. > It's not possible to put a load balancer in front of a JobManager. Only one > JM is the active master that maintains a running job. > State is also not replicated. > > Best, Fabian > > > 2018-03-19 14:24 GMT+01:00 Vishal Santoshi <vishal.santo...@gmail.com > <mailto:vishal.santo...@gmail.com>>: > Those are understandable. I am more interested in a few things ( and may be > more that could be added ) > > * As far as I can understand JM is the SPOF. Does HA become a necessity ? > * If there are 2 or more JM could we theoretically have a LB fronting them ? > Thus it is a peer to peer access ( Cassandra ) or a master slave setup for JM > HA specifically for Queryable access ( For flink jobs it is master slave ) > * Do we replicate state to other TMs for read optimization ( specifically to > avoid Hot Node issues ) ? > * If the job goes down it seems the state is not accessible. What plans to we > have to "separate concerns" for Queryable state. > > We consider Queryable State significant a feature Flink provides and would do > the necessary leg work if there are certain gaps in it being trully > considered a Highly Available Key Value store. > > Regards. > > > > > > On Mon, Mar 19, 2018 at 5:53 AM, Fabian Hueske <fhue...@gmail.com > <mailto:fhue...@gmail.com>> wrote: > Hi Vishal, > > In general, Queryable State should be ready to use. > There are a few things to consider though: > > - State queries are not synchronized with the application code, i.e., they > can happen at the same time. Therefore, the Flink application should not > modify objects that have been put into or read from the state if you are not > using the RocksDBStatebackend (which creates copies by deserialization). > - State will be rolled back after a failure. Hence, you can read writes that > are not "committed by a checkpoint". > > @Kostas, did I forget something? > > Best, Fabian > > > > 2018-03-18 16:50 GMT+01:00 Vishal Santoshi <vishal.santo...@gmail.com > <mailto:vishal.santo...@gmail.com>>: > To be more precise, is anything thing similar to > https://engineering.linkedin.com/blog/2018/03/air-traffic-controller--member-first-notifications-at-linkedin > > <https://engineering.linkedin.com/blog/2018/03/air-traffic-controller--member-first-notifications-at-linkedin> > . done in Samza, can be replicated with production level guarantees with > Flink Queryable state ( as it stands currently version 1.5 ) ? > > On Fri, Mar 16, 2018 at 5:10 PM, Vishal Santoshi <vishal.santo...@gmail.com > <mailto:vishal.santo...@gmail.com>> wrote: > We are making few decisions on use cases where Queryable state is a natural > fit > https://ci.apache.org/projects/flink/flink-docs-release-1.4/dev/stream/state/queryable_state.html > > <https://ci.apache.org/projects/flink/flink-docs-release-1.4/dev/stream/state/queryable_state.html> > > Is Queryable state production ready ? We will go to 1.5 flnk if that helps to > make the case for the usage. > > > > > > >