[ 
https://issues.apache.org/jira/browse/FLINK-3779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15307440#comment-15307440
 ] 

ASF GitHub Bot commented on FLINK-3779:
---------------------------------------

Github user uce commented on the pull request:

    https://github.com/apache/flink/pull/2051#issuecomment-222625496
  
    1. I think that you are very much looking at it with your specific use case 
in mind, which is (I think) querying other operators from within your 
operators. To me, that's not the main use case for queryable state though... 
For that, it's true that the `QueryableStateStream` does not provide much help. 
If you are already using partitioned state manually, then you will probably go 
with the `setQueryable` method, that's true. But for some state like Flink's 
internal windows, you don't have access to the `StateDescriptor` (which is 
currently not exposed for queries on the API though). Furthermore, I think it's 
good to provide a low barrier way of doing things. But if others feel the same, 
I'm certainly OK with removing it.
    
    2. I think that would be possible, yes. I agree that the 
`QueryableStateStream` is conceptually similar to a sink.
    
    3. I agree, but that was on purpose for the first version until we figure 
out how to expose it properly. You have utilities in `KvStateRequestSerializer` 
to serialize key and namespace and you can use `QueryableStateStream` to access 
the key and value serializer. Namespace is usually `VoidNamespace` if you are 
not querying windows.


> Add support for queryable state
> -------------------------------
>
>                 Key: FLINK-3779
>                 URL: https://issues.apache.org/jira/browse/FLINK-3779
>             Project: Flink
>          Issue Type: Improvement
>          Components: Distributed Runtime
>            Reporter: Ufuk Celebi
>            Assignee: Ufuk Celebi
>
> Flink offers state abstractions for user functions in order to guarantee 
> fault-tolerant processing of streams. Users can work with both 
> non-partitioned (Checkpointed interface) and partitioned state 
> (getRuntimeContext().getState(ValueStateDescriptor) and other variants).
> The partitioned state interface provides access to different types of state 
> that are all scoped to the key of the current input element. This type of 
> state can only be used on a KeyedStream, which is created via stream.keyBy().
> Currently, all of this state is internal to Flink and used in order to 
> provide processing guarantees in failure cases (e.g. exactly-once processing).
> The goal of Queryable State is to expose this state outside of Flink by 
> supporting queries against the partitioned key value state.
> This will help to eliminate the need for distributed operations/transactions 
> with external systems such as key-value stores which are often the bottleneck 
> in practice. Exposing the local state to the outside moves a good part of the 
> database work into the stream processor, allowing both high throughput 
> queries and immediate access to the computed state.
> This is the initial design doc for the feature: 
> https://docs.google.com/document/d/1NkQuhIKYmcprIU5Vjp04db1HgmYSsZtCMxgDi_iTN-g.
>  Feel free to comment.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to