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

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

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

    https://github.com/apache/flink/pull/2768#discussion_r89129366
  
    --- Diff: 
flink-core/src/main/java/org/apache/flink/api/common/state/State.java ---
    @@ -20,19 +20,30 @@
     
     import org.apache.flink.annotation.PublicEvolving;
     
    +import java.io.IOException;
    +
     /**
      * Interface that different types of partitioned state must implement.
      *
      * <p>The state is only accessible by functions applied on a 
KeyedDataStream. The key is
      * automatically supplied by the system, so the function always sees the 
value mapped to the
      * key of the current element. That way, the system can handle stream and 
state partitioning
      * consistently together.
    + *
    + * @param <T> The type of the values in the state.
      */
     @PublicEvolving
    -public interface State {
    -
    +public interface State<T> {
        /**
    -    * Removes the value mapped under the current key.
    +    * Returns the current value for the state. When the state is not
    +    * partitioned the returned value is the same for all inputs in a given
    +    * operator instance. If state partitioning is applied, the value 
returned
    +    * depends on the current operator input, as the operator maintains an
    +    * independent state for each partition.
    --- End diff --
    
    This is incorrect, we don't maintain state by partition but by key. (This 
is not your fault, it was always like this on `ValueState` and you copied it 
from there, I just discovered it now when reading your code.)


> Add get() method in State interface
> -----------------------------------
>
>                 Key: FLINK-5023
>                 URL: https://issues.apache.org/jira/browse/FLINK-5023
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>            Reporter: Xiaogang Shi
>            Assignee: Xiaogang Shi
>
> Currently, the only method provided by the State interface is `clear()`. I 
> think we should provide another method called `get()` to return the 
> structured value (e.g., value, list, or map) under the current key. 
> In fact, the functionality of `get()` has already been implemented in all 
> types of states: e.g., `value()` in ValueState and `get()` in ListState. The 
> modification to the interface can better abstract these states.



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

Reply via email to