[
https://issues.apache.org/jira/browse/FLINK-5892?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15988896#comment-15988896
]
ASF GitHub Bot commented on FLINK-5892:
---------------------------------------
Github user StefanRRichter commented on a diff in the pull request:
https://github.com/apache/flink/pull/3770#discussion_r113922612
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
---
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.checkpoint;
+
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.CompositeStateHandle;
+import org.apache.flink.runtime.state.SharedStateRegistry;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Simple container class which contains the raw/managed/legacy operator
state and key-group state handles for the sub
+ * tasks of an operator.
+ */
+public class OperatorState implements CompositeStateHandle {
+
+ private static final long serialVersionUID = -4845578005863201810L;
+
+ /** id of the operator */
+ private final OperatorID operatorID;
+
+ /** handles to non-partitioned states, subtaskindex -> subtaskstate */
+ private final Map<Integer, OperatorSubtaskState> subtaskStates;
--- End diff --
here and in a few other places in this class, we could add the `operator`
String to the variable names to make it clear for user that we are now
dealing with state on the operator level and avoid confusing flink veterans
that have a certain mental mapping for the word `(Sub)TaskState` that they must
update.
> Recover job state at the granularity of operator
> ------------------------------------------------
>
> Key: FLINK-5892
> URL: https://issues.apache.org/jira/browse/FLINK-5892
> Project: Flink
> Issue Type: New Feature
> Components: State Backends, Checkpointing
> Reporter: Guowei Ma
> Assignee: Guowei Ma
>
> JobGraph has no `Operator` info so `ExecutionGraph` can only recovery at the
> granularity of task.
> This leads to the result that the operator of the job may not recover the
> state from a save point even if the save point has the state of operator.
>
> https://docs.google.com/document/d/19suTRF0nh7pRgeMnIEIdJ2Fq-CcNVt5_hR3cAoICf7Q/edit#.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)