[
https://issues.apache.org/jira/browse/FLINK-6491?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16004909#comment-16004909
]
ASF GitHub Bot commented on FLINK-6491:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/3863#discussion_r115778815
--- Diff:
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
---
@@ -265,7 +123,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 20L: JLong,
30L: JLong), true), 1))
expectedOutput.add(new StreamRecord(
CRow(
- Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 6L: JLong,
7L: JLong), true), 2))
+ Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 7L: JLong,
7L: JLong), true), 2))
--- End diff --
The state for this record may not have been deleted before. The last input
for that key is on time `1100` and the min retention time is `2000ms`, so the
state should be kept at least until `3100` but was already discarded at `3001`
(this is the bug that I pointed out earlier).
> Add QueryConfig to specify state retention time for streaming queries
> ---------------------------------------------------------------------
>
> Key: FLINK-6491
> URL: https://issues.apache.org/jira/browse/FLINK-6491
> Project: Flink
> Issue Type: Bug
> Components: Table API & SQL
> Affects Versions: 1.3.0
> Reporter: Fabian Hueske
> Assignee: sunjincheng
> Priority: Critical
>
> By now we have a couple of streaming operators (group-windows, over-windows,
> non-windowed aggregations) that require operator state. Since state is not
> automatically cleaned-up by Flink, we need to add a mechanism to configure a
> state retention time.
> If configured, a query will retain state for a specified period of state
> inactivity. If state is not accessed within this period of time, it will be
> cleared. I propose to add two parameters for this, a min and a max retention
> time. The min retention time specifies the earliest time and the max
> retention time the latest time when state is cleared. The reasoning for
> having two parameters is that we can avoid to register many timers if we have
> more freedom when to discard state.
> This issue also introduces a QueryConfig object which can be passed to a
> streaming query, when it is emitted to a TableSink or converted to a
> DataStream (append or retraction).
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)