[
https://issues.apache.org/jira/browse/FLINK-9269?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16458775#comment-16458775
]
ASF GitHub Bot commented on FLINK-9269:
---------------------------------------
Github user sihuazhou commented on the issue:
https://github.com/apache/flink/pull/5934
Thanks for your comments, @StephanEwen , If I am not misunderstanding , we
don't need to duplicate the serializer now, because we will have a dedicated
optimization for it in the near future, I am `+1` for that. Then, what about
the concurrency problem cause by the `stateTables`, it's an obvious bug that
there could be multi thread access the `stateTab` concurrently, and one of them
could modify the `stateTab`...But so far, no users have reported that problem
yet, maybe that's because most of the user are using the `RocksDBKeyedBackend`
online instead of `HeapKeyedStateBackend`, so I think this is not an urgent
bug, but...it's still a bug, Is it should be fixed for 1.5?
> Concurrency problem in HeapKeyedStateBackend when performing checkpoint async
> -----------------------------------------------------------------------------
>
> Key: FLINK-9269
> URL: https://issues.apache.org/jira/browse/FLINK-9269
> Project: Flink
> Issue Type: Bug
> Components: State Backends, Checkpointing
> Affects Versions: 1.5.0
> Reporter: Sihua Zhou
> Assignee: Sihua Zhou
> Priority: Major
> Fix For: 1.5.0
>
>
> {code:java}
> @Nonnull
> @Override
> protected SnapshotResult<KeyedStateHandle> performOperation() throws
> Exception {
> // do something
> long[] keyGroupRangeOffsets = new
> long[keyGroupRange.getNumberOfKeyGroups()];
> for (int keyGroupPos = 0; keyGroupPos <
> keyGroupRange.getNumberOfKeyGroups(); ++keyGroupPos) {
> int keyGroupId = keyGroupRange.getKeyGroupId(keyGroupPos);
> keyGroupRangeOffsets[keyGroupPos] = localStream.getPos();
> outView.writeInt(keyGroupId);
> for (Map.Entry<String, StateTable<K, ?, ?>> kvState :
> stateTables.entrySet()) {
> // do something
> }
> }
> // do something
> }
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)