Github user rtudoran commented on a diff in the pull request:
https://github.com/apache/flink/pull/3550#discussion_r106666656
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
---
@@ -119,6 +154,66 @@ class DataStreamOverAggregate(
}
+ def createTimeBoundedProcessingTimeOverWindow(
+ inputDS: DataStream[Row]): DataStream[Row] = {
+
+ val overWindow: Group = logicWindow.groups.get(0)
+ val partitionKeys: Array[Int] = overWindow.keys.toArray
+ val namedAggregates: Seq[CalcitePair[AggregateCall, String]] =
generateNamedAggregates
+
+ val index =
overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
+ val count = input.getRowType().getFieldCount()
+ val lowerboundIndex = index - count
+ var time_boundary = 0L
+
+ logicWindow.constants.get(lowerboundIndex).getValue2 match {
+ case _: java.math.BigDecimal => time_boundary =
logicWindow.constants.get(lowerboundIndex)
+ .getValue2.asInstanceOf[java.math.BigDecimal].longValue()
+ case _ => throw new TableException("OVER Window boundaries must be
numeric")
+ }
+
+ val (aggFields, aggregates) =
AggregateUtil.transformToAggregateFunctions(
+ namedAggregates.map(_.getKey),inputType, needRetraction = false)
+
+
+ // As we it is not possible to operate neither on sliding count neither
+ // on sliding time we need to manage the eviction of the events that
+ // expire ourselves based on the proctime (system time). Therefore the
+ // current system time is assign as the timestamp of the event to be
+ // recognize by the evictor
+
+ val inputDataStreamTimed = inputDS
+ .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor())
+
+ // get the output types
+ val rowTypeInfo =
FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+
+ val result: DataStream[Row] =
+ if (partitionKeys.nonEmpty) {
+ inputDataStreamTimed.keyBy(partitionKeys:_*)
+ .window(GlobalWindows.create())
+ .trigger(CountTrigger.of(1))
+ .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
+ .apply(new
DataStreamProcTimeAggregateWindowFunction[GlobalWindow]
+ (aggregates,aggFields,inputType.getFieldCount))
+ .returns(rowTypeInfo)
+ .name(aggOpName)
+ .asInstanceOf[DataStream[Row]]
+
+ } else {
+
inputDataStreamTimed.windowAll(GlobalWindows.create()).trigger(CountTrigger.of(1))
+ .evictor(TimeEvictor.of(Time.milliseconds(time_boundary)))
+ .apply(new
DataStreamProcTimeAggregateGlobalWindowFunction[GlobalWindow](
--- End diff --
@fhueske
If you are not strict with that I would like to keep the window based
implementation. Particularly that once this is merged the next thing is that I
want to create additional JIRA issues to support Distinct aggregates for this
as well. Those for sure require having all data at once.
Regarding the support for FOLLOWING - I do not see any problems there as I
can of course work with the triggers to offer the same functionality (register
the trigger to trigger x elements/time in the future).
I do not know what capability support does processfunction provides for
evicting events.
I looked over the example
https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/process_function.html
and I do not see any eviction mechanism. This means there will be a lot of
logic and overhead moved in identifying what needs to be evicted.
All in all...I am not sure how this would look in ProcessFunction and I
have some doubts that the end result is better. However, let me know what you
think and whether you want it explicitly as a processfunction (case in which I
would like to ask if you can please point me to some examples of how to manage
eviction) or it can stay as it is.
---
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 [email protected] or file a JIRA ticket
with INFRA.
---