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

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

Github user gyfora commented on the pull request:

    https://github.com/apache/flink/pull/2051#issuecomment-222606282
  
    I will gradually add some questions/comments as I go :)
    
    1. Do we really need a QueryableStateStream exposed in the API? As you said 
this is just a pretty basic sink that anyone can probably inline who knows 
Flink enough to use the States. And I am guessing looking inside other 
operators is probably the most interesting use-case for this new feature. 
    
    (2. If we decide to keep the QueryableStateStream could it be just 
implemented as a simple RichSink? stream.addSink(...) instead of adding another 
operator to the runtime layer)
    
     3. I think it would be great to make the KvStateClient somehow aware of 
the types, because passing keynamespace byte arrays will be confusing for users 
as they will have no idea what the namespace is. The namespaces are pretty 
internal to the system. Maybe we could allow users to register serializers for 
state ids in the KvStateClient so they would not need to manually pass byte 
arrays and they could work with the actual keys. Also if they don't want to 
specify the namespace, we could then serialize it with the VoidSerializer as it 
usually happens in the runtime in most cases.


> 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