[ 
https://issues.apache.org/jira/browse/FLINK-4844?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15582782#comment-15582782
 ] 

ASF GitHub Bot commented on FLINK-4844:
---------------------------------------

Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2648#discussion_r83673478
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SnapshotInProgressSubtaskState.java
 ---
    @@ -0,0 +1,81 @@
    +/*
    + * 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.state.KeyGroupsStateHandle;
    +import org.apache.flink.runtime.state.OperatorStateHandle;
    +
    +import java.util.concurrent.RunnableFuture;
    +
    +/**
    + * Encapsulates all runnable futures draw snapshots for a single subtask 
state of an in-flight checkpointing operation.
    + */
    +public class SnapshotInProgressSubtaskState {
    --- End diff --
    
    I think this could be changed to
    ```
    /**
     * Result of {@link AbstractStreamOperator#snapshotState}.
     */
    public class OperatorSnapshotResult { ... }
    ```
    
    to make it more clearer what it is supposed to be. And it should probably 
be in the same module/package as `AbstractStreamOperator` but the code layout 
of the state classes seems a bit messy so not sure if it's possible.



> Partitionable Raw Keyed/Operator State
> --------------------------------------
>
>                 Key: FLINK-4844
>                 URL: https://issues.apache.org/jira/browse/FLINK-4844
>             Project: Flink
>          Issue Type: New Feature
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>
> Partitionable operator and keyed state are currently only available by using 
> backends. However, the serialization code for many operators is build around 
> reading/writing their state to a stream for checkpointing. We want to provide 
> partitionable states also through streams, so that migrating existing 
> operators becomes more easy.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to