[
https://issues.apache.org/jira/browse/FLINK-7213?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16126153#comment-16126153
]
ASF GitHub Bot commented on FLINK-7213:
---------------------------------------
Github user StephanEwen commented on a diff in the pull request:
https://github.com/apache/flink/pull/4353#discussion_r133016663
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java
---
@@ -75,31 +103,79 @@
*/
private final long stateSize;
+ @VisibleForTesting
+ public OperatorSubtaskState(StreamStateHandle legacyOperatorState) {
+
+ this(legacyOperatorState,
+ Collections.<OperatorStateHandle>emptyList(),
+ Collections.<OperatorStateHandle>emptyList(),
+ Collections.<KeyedStateHandle>emptyList(),
+ Collections.<KeyedStateHandle>emptyList());
+ }
+
+ /**
+ * Empty state.
+ */
+ public OperatorSubtaskState() {
--- End diff --
Minor optimization: One could make this constructor `private` and have a
field `OperatorSubtaskState.EMPTY` as a placeholder for the empty states. I'd
leave this to you whether you think it worth doing...
> Introduce state management by OperatorID in TaskManager
> -------------------------------------------------------
>
> Key: FLINK-7213
> URL: https://issues.apache.org/jira/browse/FLINK-7213
> Project: Flink
> Issue Type: Improvement
> Components: State Backends, Checkpointing
> Affects Versions: 1.4.0
> Reporter: Stefan Richter
> Assignee: Stefan Richter
>
> Flink-5892 introduced the job manager / checkpoint coordinator part of
> managing state on the operator level instead of the task level by introducing
> explicit operator_id -> state mappings. However, this explicit mapping was
> not introduced in the task manager side, so the explicit mapping is still
> converted into a mapping that suits the implicit operator chain order.
> We should also introduce explicit operator ids to state management on the
> task manager.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)