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

    https://github.com/apache/flink/pull/1305#discussion_r45373479
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java ---
    @@ -64,33 +64,35 @@
        /**
         * Closes the state backend, releasing all internal resources, but does 
not delete any persistent
         * checkpoint data.
    -    * 
    +    *
         * @throws Exception Exceptions can be forwarded and will be logged by 
the system
         */
        public abstract void close() throws Exception;
    -   
    +
        // 
------------------------------------------------------------------------
        //  key/value state
        // 
------------------------------------------------------------------------
     
        /**
         * Creates a key/value state backed by this state backend.
    -    * 
    +    *
    +    * @param operatorId Unique id for the operator creating the state
    +    * @param stateName Name of the created state
         * @param keySerializer The serializer for the key.
         * @param valueSerializer The serializer for the value.
         * @param defaultValue The value that is returned when no other value 
has been associated with a key, yet.
         * @param <K> The type of the key.
         * @param <V> The type of the value.
    -    * 
    +    *
         * @return A new key/value state backed by this backend.
    -    * 
    +    *
         * @throws Exception Exceptions may occur during initialization of the 
state and should be forwarded.
         */
    -   public abstract <K, V> KvState<K, V, Backend> createKvState(
    +   public abstract <K, V> KvState<K, V, Backend> createKvState(int 
operatorId, String stateName,
    --- End diff --
    
    The "name" (as a string) of the state is a very API specific thing that no 
other part of the runtime is concerned with. The operator ID is something 
specific to the StreamGraphBuilder and not to the streaming tasks at all. I 
think we are tying things together here that should not be tied together.
    
    I still do not understand how this affects sharding. Does the shard 
assignment depend on the state name (rather than the parallel subtask / 
JobVertexId) ?
    
    I only see that the table names will have the task name instead of the name 
of the state.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to