[
https://issues.apache.org/jira/browse/FLINK-2677?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14933254#comment-14933254
]
ASF GitHub Bot commented on FLINK-2677:
---------------------------------------
Github user aljoscha commented on the pull request:
https://github.com/apache/flink/pull/1184#issuecomment-143735489
One thing to note here is that you cannot use the old `TriggerPolicy` and
`EvictionPolicy` anymore. The old concept of windowing helpers is replaced by
windowing policies. There is `Time`, `Count` and `Delta`. These are translated
to `Trigger` and `Evictor` instances using a fixed translation mapping in
`PolicyToOperator`.
If a user wan't the full flexibility they have to specify a
`WindowAssigner`. For example, this old API code:
```java
input.window(new MyCustomEvictionPolicy).every(new MyCustomTriggerPolicy)
```
can be written with the new API as:
```java
input.window(GlobalWindows.create())
.trigger(new MyCustomTrigger)
.evictor(new MyCustomEvictor)
```
(I could add a shortcut to `KeyedDataStream`, so that you can directly do
`.window(Evictor, Trigger)` with an implied `GlobalWindows` assigner.)
Also worth mentioning is that there is no pre-aggregation if an `Evictor`
is used.
> Add a general-purpose keyed-window operator
> -------------------------------------------
>
> Key: FLINK-2677
> URL: https://issues.apache.org/jira/browse/FLINK-2677
> Project: Flink
> Issue Type: Sub-task
> Components: Streaming
> Affects Versions: 0.10
> Reporter: Stephan Ewen
> Assignee: Aljoscha Krettek
> Fix For: 0.10
>
>
> This operator should support:
> - Customizable triggers
> - Eviction on triggers: all / none / custom
> - Discard by time (expiry of state)
> - Event time time window assignment
> This set of requirements is effectively a mix between the current
> trigger/evict model and the Cloud Dataflow window definition model.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)