flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From pnowojski <...@git.apache.org>
Subject [GitHub] flink pull request #5239: [FLINK-8360] Implement task-local state recovery
Date Thu, 11 Jan 2018 14:26:20 GMT
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5239#discussion_r160929798
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManager.java
---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.state;
    +
    +import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
    +import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
    +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
    +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    +import org.apache.flink.runtime.jobgraph.OperatorID;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +/**
    + * This interface provides methods to report and retrieve state for a task.
    + *
    + * <p>When a checkpoint or savepoint is triggered on a task, it will create snapshots
for all stream operator instances
    + * it owns. All operator snapshots from the task are then reported via this interface.
A typical implementation will
    + * dispatch and forward the reported state information to interested parties such as
the checkpoint coordinator or a
    + * local state store.
    + *
    + * <p>This interface also offers the complementary method that provides access
to previously saved state of operator
    + * instances in the task for restore purposes.
    + */
    +public interface TaskStateManager extends CheckpointListener {
    --- End diff --
    
    Disclaimer, despite this not being a nit, if you have strong objections I will not fight
over this comment :)
    
    I think this class should be split into two:
    1. `TaskStateReporter` that handles sending the messages to the master (extracting and
maybe wrapping `checkpointResponder` from this one)
    2. and already existing `TaskLocalStateStore`
    
    and use them separately.  `StreamTask` would call:
    ```
    localStateStore.store(...)
    taskStateReporter.reportTaskStateSnapshots(...)
    ```
    
    And all of the remaining usages of the `TaskStateManager` could be replaced by calls to
the `TaskLocalStateStore` directly. 
    
    One notable exception might be `org.apache.flink.runtime.state.TaskStateManagerImpl#prioritizedOperatorState`
which is not present in this commit, but comes to live later on. It could either be converted
into static function (since it doesn't seem to have any side effects) taking three parameters
 `(jobManagerTaskRestore, localStateStore, operatorId)`  or create some third entity `TaskStateRestoreSelector`,
`LocalAndRemoteTaskStateSelector`, `OperatorStatePrioritizer` or something else.
    
    I think that would let us get rid of one vague `Manager` class and replace it with some
smaller purpose build entities, which might be handy both for code understanding, testing
and refactoring in the future.


---

Mime
View raw message