You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/02/03 16:41:51 UTC

[jira] [Commented] (FLINK-5024) Add SimpleStateDescriptor to clarify the concepts

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

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

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

    https://github.com/apache/flink/pull/3243#discussion_r99374243
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java ---
    @@ -20,34 +20,29 @@
     
     import org.apache.flink.annotation.PublicEvolving;
     import org.apache.flink.api.common.ExecutionConfig;
    -import org.apache.flink.api.common.typeinfo.TypeInformation;
    -import org.apache.flink.api.common.typeutils.TypeSerializer;
    -import org.apache.flink.api.java.typeutils.TypeExtractor;
    -import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    -import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
     import org.apache.flink.util.Preconditions;
     
    -import java.io.ByteArrayInputStream;
    -import java.io.ByteArrayOutputStream;
    -import java.io.IOException;
    -import java.io.ObjectInputStream;
    -import java.io.ObjectOutputStream;
     import java.io.Serializable;
     
     import static java.util.Objects.requireNonNull;
     
     /**
    - * Base class for state descriptors. A {@code StateDescriptor} is used for creating partitioned
    - * {@link State} in stateful operations. This contains the name and can create an actual state
    - * object given a {@link StateBackend} using {@link #bind(StateBackend)}.
    - *
    + * Base class for state descriptors. A {@code StateDescriptor} is used for creating
    + * {@link State keyed state} in stateful operations. The descriptor contains the name of the state,
    --- End diff --
    
    Please close this issue as well if you address this here: https://issues.apache.org/jira/browse/FLINK-5599


> Add SimpleStateDescriptor to clarify the concepts
> -------------------------------------------------
>
>                 Key: FLINK-5024
>                 URL: https://issues.apache.org/jira/browse/FLINK-5024
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>            Reporter: Xiaogang Shi
>            Assignee: Xiaogang Shi
>
> Currently, StateDescriptors accept two type arguments : the first one is the type of the created state and the second one is the type of the values in the states. 
> The concepts however is a little confusing here because in ListStates, the arguments passed to the StateDescriptors are the types of the list elements instead of the lists. It also makes the implementation of MapStates difficult.
> I suggest not to put the type serializer in StateDescriptors, making StateDescriptors independent of the data structures of the values. 
> A new type of StateDescriptor named SimpleStateDescriptor can be provided to abstract those states (namely ValueState, ReducingState and FoldingState) whose states are not composited. 
> The states (e.g. ListStates and MapStates) can implement their own descriptors according to their data structures. 



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