[
https://issues.apache.org/jira/browse/FLINK-9486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16537063#comment-16537063
]
ASF GitHub Bot commented on FLINK-9486:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/6276#discussion_r201016373
--- Diff:
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
---
@@ -394,6 +407,16 @@ public void dispose() {
}
}
+ @Override
+ public <T extends HeapPriorityQueueElement>
KeyGroupedInternalPriorityQueue<T> create(
+ @Nonnull String stateName,
+ @Nonnull TypeSerializer<T> byteOrderedElementSerializer,
+ @Nonnull Comparator<T> elementComparator,
+ @Nonnull KeyExtractorFunction<T> keyExtractor) {
--- End diff --
Indentation
> Introduce TimerState in keyed state backend
> -------------------------------------------
>
> Key: FLINK-9486
> URL: https://issues.apache.org/jira/browse/FLINK-9486
> Project: Flink
> Issue Type: Sub-task
> Components: State Backends, Checkpointing
> Reporter: Stefan Richter
> Assignee: Stefan Richter
> Priority: Major
> Labels: pull-request-available
> Fix For: 1.6.0
>
>
> This is the first implementation subtask.
> Goal of this PR is to introduce a timer state that is registered with the
> keyed state backend, similar to other forms of keyed state.
> For the {{HeapKeyedStateBackend}}, this state lives on the same level as the
> {{StateTable}} that hold other forms of keyed state, and the implementation
> is basically backed by {{InternalTimerHeap}}.
> For {{RocksDBKeyedStateBackend}}, in this first step, we also introduce this
> state, outside of RocksDB and based upon {{InternalTimerHeap}}. This is an
> intermediate state, and we will later also implement the alternative to store
> the timers inside a column families in RocksDB. However, by taking this step,
> we could also still offer the option to have RocksDB state with heap-based
> timers.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)