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

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

GitHub user bowenli86 opened a pull request:

    https://github.com/apache/flink/pull/5300

    [FLINK-8411] [State Backends] inconsistent behavior between 
HeapListState#add() and RocksDBListState#add()

    ## What is the purpose of the change
    
    `HeapListState#add(null)` will result in the whole state being cleared or 
wiped out. There's never a unit test for `List#add(null)` in 
`StateBackendTestBase`
    
    ## Brief change log
    
    - changed ListState impls such that `add(null)` will be explicitly ignored
    - added unit tests to test `add(null)`
    - updated javaDoc
    
    ## Verifying this change
    
    This change is already covered by existing tests, such as 
`StateBackendTestBase`.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: yes
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (JavaDocs)
    
    
    Note! **This work depends on FLINK-7983 and its PR at 
https://github.com/apache/flink/pull/5281**
    
    cc @StefanRRichter 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/bowenli86/flink FLINK-8411

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5300.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5300
    
----
commit b4034f67e727fef68740221e3b31cd131c905df1
Author: Bowen Li <bowenli86@...>
Date:   2018-01-02T19:21:28Z

    update local branch

commit 11e9c255cd51304c5281d55226fbb6fe8360d8a2
Author: Bowen Li <bowenli86@...>
Date:   2018-01-04T01:35:11Z

    remove sh

commit 72edff2a82df625203289e4b5be23db36b03abe3
Author: Bowen Li <bowenli86@...>
Date:   2018-01-12T08:14:55Z

    [FLINK-7938] introduce addAll() to ListState

commit 138d6f63dff7a840b716bf8900f97940f7d61dd8
Author: Bowen Li <bowenli86@...>
Date:   2018-01-12T18:56:42Z

    add unit tests

commit 481a5a98b7b655658855e18023c7c28328b0b47d
Author: Bowen Li <bowenli86@...>
Date:   2018-01-12T19:03:07Z

    add documentation for addAll()

commit cced3bac65660b27e17d258b3fd3880e9571bcf6
Author: Bowen Li <bowenli86@...>
Date:   2018-01-13T23:06:11Z

    add more cases in unit test

commit 18fb3ff2965653dc0bd7c7a2d7a419ce8c7c6e8a
Author: Bowen Li <bowenli86@...>
Date:   2018-01-15T23:24:19Z

    [FLINK-8411] inconsistent behavior between HeapListState#add() and 
RocksDBListState#add()

----


> inconsistent behavior between HeapListState#add() and RocksDBListState#add()
> ----------------------------------------------------------------------------
>
>                 Key: FLINK-8411
>                 URL: https://issues.apache.org/jira/browse/FLINK-8411
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.4.0
>            Reporter: Bowen Li
>            Assignee: Bowen Li
>            Priority: Critical
>             Fix For: 1.5.0, 1.4.1
>
>
> You can see that {{HeapListState#add(null)}} will result in the whole state 
> being cleared or wiped out. There's never a unit test for {{List#add(null)}} 
> in {{StateBackendTestBase}}
> {code:java}
> // HeapListState
> @Override
>       public void add(V value) {
>               final N namespace = currentNamespace;
>               if (value == null) {
>                       clear();
>                       return;
>               }
>               final StateTable<K, N, ArrayList<V>> map = stateTable;
>               ArrayList<V> list = map.get(namespace);
>               if (list == null) {
>                       list = new ArrayList<>();
>                       map.put(namespace, list);
>               }
>               list.add(value);
>       }
> {code}
> {code:java}
> // RocksDBListState
> @Override
>       public void add(V value) throws IOException {
>               try {
>                       writeCurrentKeyWithGroupAndNamespace();
>                       byte[] key = keySerializationStream.toByteArray();
>                       keySerializationStream.reset();
>                       DataOutputViewStreamWrapper out = new 
> DataOutputViewStreamWrapper(keySerializationStream);
>                       valueSerializer.serialize(value, out);
>                       backend.db.merge(columnFamily, writeOptions, key, 
> keySerializationStream.toByteArray());
>               } catch (Exception e) {
>                       throw new RuntimeException("Error while adding data to 
> RocksDB", e);
>               }
>       }
> {code}
> The fix should correct the behavior to be consistent between the two state 
> backends, as well as adding a unit test for {{ListState#add(null)}}. For the 
> correct behavior, I believe adding null with {{add(null)}} should simply be 
> ignored without any consequences.
> cc [~srichter]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to