[
https://issues.apache.org/jira/browse/FLINK-3779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15388058#comment-15388058
]
ASF GitHub Bot commented on FLINK-3779:
---------------------------------------
Github user uce commented on the issue:
https://github.com/apache/flink/pull/2051
Regarding local vs. cluster mode: that's on purpose, but we can certainly
change that behaviour. For now, you would have to run in cluster mode.
Regarding the serializer: assuming that it is a Flink `Tuple2<Long,
String>` you can use the following to get the serializer:
```java
TypeSerializer<?>[] fieldSerializers = new TypeSerializer[] {
StringSerializer.INSTANCE,
LongSerializer.INSTANCE
};
TypeSerializer<Tuple2<String, Long>> serializer = new TupleSerializer<>(
(Class<Tuple2<String, Long>>) (Class<?>) Tuple2.class,
fieldSerializers);
```
**Just to make sure that we are on the same page: the state of this PR is
not the final queryable state API, but only the initial low-level version.**
Really looking forward to further feedback. Thank you for trying it out at this
stage. :-)
> Add support for queryable state
> -------------------------------
>
> Key: FLINK-3779
> URL: https://issues.apache.org/jira/browse/FLINK-3779
> Project: Flink
> Issue Type: Improvement
> Components: Distributed Coordination
> 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)