flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Aljoscha Krettek (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8411) HeapListState#add(null) will wipe out entire list state
Date Tue, 30 Jan 2018 09:53:00 GMT

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

Aljoscha Krettek commented on FLINK-8411:
-----------------------------------------

[~phoenixjiangnan] & [~srichter] I thought about this some more. I don't think we can
leave the current behaviour for 1.4.1 because it is a change in semantics that might break
existing code.

The fact that the RocksDB backend and the Heap backend behaved differently before is not good
but I think the previous RocksDB behaviour might be the preferable behaviour: if the serialiser
supports it adding {{null}} to the list might be a valid thing for some users. Simply ignoring
{{nulls}} might lead to unexpected consequences.

What do you think?

> 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: 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)

Mime
View raw message