Thanks for bringing this up again Vino.

Considering the current status that the community is lack of committer
bandwidth on QueryableState [1] [2], I think David has made a good
point/concern, that we need to collect more information about valid
in-production QueryableState use cases to draw more attention on this
module (I have collected some, and there are indeed scattered requirements
around QueryableState, but let's open one thread to explicitly discuss
about this, later). IMHO this is the precondition of putting review
resource to improve the module.

I believe there will be committer bandwidth on this, sooner or later, but
we need to go further by ourselves before that happens, and what we need is
more patience. Hope this won't frustrate you too much, and please correct
me if any committer would like to put more effort into this. Thanks.

[1] https://s.apache.org/MaOl
[2] https://s.apache.org/r8k8a

Best Regards,
Yu


On Fri, 25 Oct 2019 at 23:59, vino yang <yanghua1...@gmail.com> wrote:

> Hi David,
>
> I know that in some scenarios, the queryable state has some limitations.
> But in some scenarios, it is valuable, such as in scenarios where you need
> to observe "trends." It can be analogized to the two existing features
> provided by Flink web UI:
>
>    - Metrics: An instantaneous value that can be queried is like a
>    business metric of Flink jobs;
>    - BackPressure: A state query is like instantaneous sampling, and a
>    periodic query is similar to periodic sampling;
>
> I just want to say that although our state can get the semantic support of
> "correctness", however, users don't always pay attention to (or use) the
> final correct result. Sometimes they want to observe the trend of the value
> of a key in one job run cycle. When the job is resumed, then the job goes
> into another new run cycle, and they can continue to observe its trend.
>
> The queryable state is oriented to KV State, so there is no problem with
> point-oriented queries. Several months ago, I have also suggested whether
> the window state can be queried [1]. But this topic has not attracted
> attention, maybe everyone is busy releasing 1.9 at that moment. I see there
> was a project named "query-window-example" in dataArtisans try to take an
> effort to do this job.[2]
>
> There was another ML thread which asked "Making broadcast state
> queryable?".[3]
>
> So, I really want to know what does the community thinks about the
> queryable state.
>
> If the community feels that it still has some meaning, then we can
> continue to improve it and see how it will work in the future.
>
> Best,
> Vino
>
> [1]:
> http://mail-archives.apache.org/mod_mbox/flink-user/201907.mbox/%3CCAA_=o7bn3nkbsoeayfhxpdteeffxdswzrp2wpyjwp9mnplz...@mail.gmail.com%3E
> [2]: https://github.com/dataArtisans/query-window-example
> [3]:
> http://mail-archives.apache.org/mod_mbox/flink-user/201909.mbox/%3CCAM7-19KYbu0c_==npts9z+ckgycpzqp+i5plgxg6xqaqdn5...@mail.gmail.com%3E
>
> David Anderson <da...@ververica.com> 于2019年10月25日周五 下午4:56写道:
>
>> I've encountered a number of Flink users who considered using
>> queryable state, but after investigation, decided not to. The reasons
>> have been:
>>
>> (1) The current interface (point queries fetching state for one key)
>> is too limiting. What some folks really want/need is the ability to
>> execute SQL-queries against the state.
>>
>> (2) The state is not highly available. If the job isn't running, the
>> state can not be queried. (Hypothetically, a queryable state service
>> could fall back to querying a snapshot for state for a job that isn't
>> currently running, but that sounds a bit crazy.)
>>
>> (3) During recovery the state can regress, in the sense that it
>> reflects an earlier point in time than what may've been previously
>> fetched.
>>
>> (4) The state that is wanted (e.g., window state, or operator state)
>> isn't queryable.
>>
>> Best,
>> David
>>
>> On Fri, Oct 25, 2019 at 9:51 AM vino yang <yanghua1...@gmail.com> wrote:
>> >
>> > Hi Jiayi,
>> >
>> > Thanks for your valuable feedback and suggestions.
>> >
>> > In our production env, we still have many applications wrote by
>> DataStream
>> > API.
>> >
>> > Currently, we have some requirements that require Adhoc query for the
>> > runtime Flink job. The existing query interface is very difficult to
>> use.
>> > This improvement is to enhance the usability of the queryable state.
>> > Currently, I only limit its boundaries to improved design. It does not
>> > involve the exploration of the scope of capabilities and use scenarios.
>> Of
>> > course, these directions are very interesting and I hope to think
>> further.
>> >
>> > As you said, the queryable state and state-processing-api are used to
>> > handle state. From this perspective, they seem to be able to merge or
>> > integrate modules in some way (to avoid the fast-expanding Flink
>> modules).
>> >
>> >
>> > However, the queryable state queries the online KV state at a certain
>> point
>> > in time, while the state-processing-api uses the DataSet API to read,
>> > write, and analyze the offline savepoint. There is a big difference.
>> >
>> > How to look at the online queryable state and the offline
>> > state-processing-api may require further discussion in the community.
>> So, I
>> > pinged people who might be related to these modules to get more valuable
>> > feedback.
>> >
>> > Best,
>> > Vino
>> >
>> >
>> >
>> > bupt_ljy <bupt_...@163.com> 于2019年10月24日周四 下午9:04写道:
>> >
>> > > Hi vino,
>> > >
>> > > +1 for improvement on queryable state feature. This reminds me of the
>> > > state-processing-api module, which is very helpful when we analyze
>> state in
>> > > offline. However currently we don’t have many ways to know what is
>> > > happening about the state inside a running application, which makes
>> me feel
>> > > that this has a good potential. Since these two modules are seperate
>> but
>> > > doing the similar work(anaylyzing state), maybe we have to think more
>> about
>> > > their orientation, or maybe integrate them in a graceful way in the
>> future.
>> > >
>> > > Anyway, this is a great work and it’d be better if we can hear more
>> > > thoughts and use cases.
>> > >
>> > > Best Regards,
>> > > Jiayi Liao
>> > >
>> > >  Original Message
>> > > *Sender:* vino yang<yanghua1...@gmail.com>
>> > > *Recipient:* dev@flink.apache.org<dev@flink.apache.org>
>> > > *Date:* Tuesday, Oct 22, 2019 15:42
>> > > *Subject:* [DISCUSS] Introduce a location-oriented two-stage query
>> > > mechanism toimprove the queryable state.
>> > >
>> > > Hi guys,
>> > >
>> > >
>> > > Currently, queryable state's client is hard to use. Because it
>> requires
>> > > users to know the address of TaskManager and the port of the proxy.
>> > > Actually, most users who do not have good knowledge about the Flink's
>> inner
>> > > and runtime in production. The queryable state clients directly
>> interact
>> > > with query state client proxies which host on each TaskExecutor. This
>> > > design requires users to know too much detail.
>> > >
>> > >
>> > >
>> > > We introduce a location service component to improve the architecture
>> of
>> > > the queryable state and hide the details of the task executors. We
>> first
>> > > give a brief introduction to our design in Section 2 and then detail
>> the
>> > > implementation in Section 3. At last, we describe some future work
>> that can
>> > > be done.
>> > >
>> > >
>> > > [image: Screen Shot 2019-10-22 at 10.05.11 AM.png]
>> > >
>> > >
>> > > I have given an initialized implementation in my Flink repository[2].
>> One
>> > > thing that needs to be stated is that we have not changed the existing
>> > > solution, so it still works according to the previous modes.
>> > >
>> > > The design documentation is here[3].
>> > >
>> > > Any suggestion and feedback are welcome and appriciated.
>> > >
>> > > [1]: https://statefun.io/
>> > > [2]:
>> https://github.com/yanghua/flink/tree/improve-queryable-state-master
>> > > [3]:
>> > >
>> https://docs.google.com/document/d/181qYVIiHQGrc3hCj3QBn1iEHF4bUztdw4XO8VSaf_uI/edit?usp=sharing
>> > >
>> > > Best,
>> > > Vino
>> > >
>>
>

Reply via email to