[jira] [Created] (FLINK-5991) Expose Broadcast Operator State through public APIs

classic Classic list List threaded Threaded
1 message Options
Reply | Threaded
Open this post in threaded view
|

[jira] [Created] (FLINK-5991) Expose Broadcast Operator State through public APIs

Shang Yuanchun (Jira)
Tzu-Li (Gordon) Tai created FLINK-5991:
------------------------------------------

             Summary: Expose Broadcast Operator State through public APIs
                 Key: FLINK-5991
                 URL: https://issues.apache.org/jira/browse/FLINK-5991
             Project: Flink
          Issue Type: New Feature
          Components: State Backends, Checkpointing, Streaming
            Reporter: Tzu-Li (Gordon) Tai
            Assignee: Tzu-Li (Gordon) Tai
             Fix For: 1.3.0


The broadcast operator state functionality was added in FLINK-5265, it just hasn't been exposed through any public APIs yet.

Currently, we have 2 streaming connector features for 1.3 that are pending on broadcast state: rescalable Kinesis / Kafka consumers with shard / partition discovery (FLINK-4821 & FLINK-4022). We should consider exposing broadcast state for the 1.3 release also.

This JIRA also serves the purpose to discuss how we want to expose it.

To initiate the discussion, I propose:

1. For the more powerful `CheckpointedFunction`, add:
{code}
<S> ListState<S> getBroadcastOperatorState(ListStateDescriptor<S> stateDescriptor);

<T extends Serializable> ListState<T> getBroadcastSerializableListState(String stateName);
{code}

2. For a simpler `ListCheckpointed` variant, we probably should have a separate `BroadcastListCheckpointed` interface.

Extending `ListCheckpointed` to let the user define either `PARTITIONABLE` or `BROADCAST` list used might also be possible, if we can rely on a contract that the value doesn't change. Or we expose a defining method (e.g. `getListStateType()`) that is called only once in the operator.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)