Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3919#discussion_r116733901
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/CountTriggerWithCleanupState.scala
 ---
    @@ -0,0 +1,146 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.flink.table.runtime.triggers
    +
    +import java.lang.{Long => JLong}
    +
    +import org.apache.flink.api.common.functions.ReduceFunction
    +import org.apache.flink.api.common.state._
    +import 
org.apache.flink.streaming.api.windowing.triggers.Trigger.TriggerContext
    +import org.apache.flink.streaming.api.windowing.triggers.{Trigger, 
TriggerResult}
    +import org.apache.flink.streaming.api.windowing.windows.Window
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import 
org.apache.flink.table.runtime.triggers.CountTriggerWithCleanupState.Sum
    +
    +class CountTriggerWithCleanupState[W <: Window](queryConfig: 
StreamQueryConfig, maxCount: Long)
    +  extends Trigger[Any, W] {
    +
    +  private val serialVersionUID: Long = 1L
    +
    +  protected val minRetentionTime: Long = 
queryConfig.getMinIdleStateRetentionTime
    +  protected val maxRetentionTime: Long = 
queryConfig.getMaxIdleStateRetentionTime
    +  protected val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  private val stateDesc: ReducingStateDescriptor[JLong] =
    +    new ReducingStateDescriptor[JLong]("count", new Sum, Types.LONG)
    +
    +  private val cleanupStateDesc: ValueStateDescriptor[JLong] =
    +    new ValueStateDescriptor[JLong]("countCleanup", Types.LONG)
    +
    +  override def canMerge: Boolean = true
    +
    +  override def onMerge(window: W, ctx: Trigger.OnMergeContext) {
    +    ctx.mergePartitionedState(stateDesc)
    --- End diff --
    
    This will fail because `ValueState` is not a `MergingState`. We could 
implement the `cleanupStateDesc` as a `ReducingState` with a `max` 
`ReduceFunction`. That would allow the Flink to merge the state.
    
    On the other hand, we won't use the Trigger in merging windows anyways, to 
it would also be fine to not support merging at all (`canMerge = false`). I'd 
actually go for this option because it simplifies the trigger.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to