[
https://issues.apache.org/jira/browse/FLINK-7731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16198617#comment-16198617
]
Aljoscha Krettek commented on FLINK-7731:
-----------------------------------------
[~gerardg] Only cleaning up when returning {{FIRE_AND_PURGE}} from
{{onProcessingTime()}} should actually work. The reason why it doesn't work
could be that the check does not work properly. Instead of
{code}
if (ctx.getPartitionedState(latestDeadlineDesc).value() != time) {
{code}
could you try using
{code}
if (!ctx.getPartitionedState(latestDeadlineDesc).value().equals(time)) {
{code}
> Trigger on GlobalWindow does not clean state completely
> -------------------------------------------------------
>
> Key: FLINK-7731
> URL: https://issues.apache.org/jira/browse/FLINK-7731
> Project: Flink
> Issue Type: Bug
> Components: Core, DataStream API
> Affects Versions: 1.3.2
> Reporter: Gerard Garcia
> Priority: Minor
>
> I have an operator that consists of:
> CoGroup Datastream -> GlobalWindow -> CustomTrigger -> Apply function
> The custom trigger fires and purges the elements after it has received the
> expected number of elements (or when a timeout fires) from one of the streams
> and the apply function merges the elements with the ones received from the
> other stream. It appears that the state of the operator grows continuously so
> it seems it never gets completely cleaned.
> There is a discussion in
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Clean-GlobalWidnow-state-td15613.html
> that suggests that it may be a bug.
> This job reproduces the issue:
> https://github.com/GerardGarcia/flink-global-window-growing-state
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)