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

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

GitHub user aljoscha opened a pull request:

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

    [FLINK-8411] Don't allow null in ListState.add()/addAll()

    R: @StefanRRichter, @bowenli86 
    
    It turns out that this is a bit trickier than assumed earlier: 
`ListState.addAll()` was not considered and also had inconsistent behaviour 
between state backends before.

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

    $ git pull https://github.com/aljoscha/flink jira-8411-fix-list-add

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

    https://github.com/apache/flink/pull/5485.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 #5485
    
----
commit b39aa20d60df9effb536cc96c308645b9688113d
Author: Aljoscha Krettek <aljoscha.krettek@...>
Date:   2018-02-14T11:04:20Z

    [FLINK-8411] Don't allow null in ListState.add()/addAll()

----


> HeapListState#add(null) will wipe out entire list state
> -------------------------------------------------------
>
>                 Key: FLINK-8411
>                 URL: https://issues.apache.org/jira/browse/FLINK-8411
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.4.0
>            Reporter: Bowen Li
>            Assignee: Bowen Li
>            Priority: Blocker
>             Fix For: 1.5.0
>
>
> 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