[
https://issues.apache.org/jira/browse/FLINK-8715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16456238#comment-16456238
]
ASF GitHub Bot commented on FLINK-8715:
---------------------------------------
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/5885#discussion_r184657398
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapFoldingState.java
---
@@ -49,17 +49,18 @@
/**
* Creates a new key/value state for the given hash map of key/value
pairs.
*
- * @param stateDesc The state identifier for the state. This contains
name
- * and can create a default state value.
+ * @param valueSerializer The serializer for the state.
* @param stateTable The state tab;e to use in this kev/value state.
May contain initial state.
*/
public HeapFoldingState(
- FoldingStateDescriptor<T, ACC> stateDesc,
StateTable<K, N, ACC> stateTable,
TypeSerializer<K> keySerializer,
- TypeSerializer<N> namespaceSerializer) {
- super(stateDesc, stateTable, keySerializer,
namespaceSerializer);
- this.foldTransformation = new FoldTransformation<>(stateDesc);
+ TypeSerializer<ACC> valueSerializer,
+ TypeSerializer<N> namespaceSerializer,
+ ACC defaultValue,
--- End diff --
Javadocs for all state classes are now updated.
> RocksDB does not propagate reconfiguration of serializer to the states
> ----------------------------------------------------------------------
>
> Key: FLINK-8715
> URL: https://issues.apache.org/jira/browse/FLINK-8715
> Project: Flink
> Issue Type: Bug
> Components: State Backends, Checkpointing
> Affects Versions: 1.3.2
> Reporter: Arvid Heise
> Assignee: Tzu-Li (Gordon) Tai
> Priority: Blocker
> Fix For: 1.5.0
>
>
> Any changes to the serializer done in #ensureCompability are lost during the
> state creation.
> In particular,
> [https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBValueState.java#L68]
> always uses a fresh copy of the StateDescriptor.
> An easy fix is to pass the reconfigured serializer as an additional parameter
> in
> [https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java#L1681]
> , which can be retrieved through the side-output of getColumnFamily
> {code:java}
> kvStateInformation.get(stateDesc.getName()).f1.getStateSerializer()
> {code}
> I encountered it in 1.3.2 but the code in the master seems unchanged (hence
> the pointer into master). I encountered it in ValueState, but I suspect the
> same issue can be observed for all kinds of RocksDB states.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)