[
https://issues.apache.org/jira/browse/FLINK-3243?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15137101#comment-15137101
]
ASF GitHub Bot commented on FLINK-3243:
---------------------------------------
Github user rmetzger commented on a diff in the pull request:
https://github.com/apache/flink/pull/1513#discussion_r52185124
--- Diff:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
---
@@ -163,7 +166,11 @@ public KeyedStream(DataStream<T> dataStream,
KeySelector<T, KEY> keySelector, Ty
* @param size The size of the window.
*/
public WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size)
{
- return window(TumblingTimeWindows.of(size));
+ if (environment.getStreamTimeCharacteristic() ==
TimeCharacteristic.PROCESSING_TIME) {
--- End diff --
I'm not sure how the rest of the DataStream API is behaving, but I though
that it doesn't matter when the environment settings (for example parallelism)
are set.
In this case, users have to set the `TimeCharacteristic` before using
operators.
> Fix Interplay of TimeCharacteristic and Time Windows
> ----------------------------------------------------
>
> Key: FLINK-3243
> URL: https://issues.apache.org/jira/browse/FLINK-3243
> Project: Flink
> Issue Type: Bug
> Components: Streaming
> Affects Versions: 1.0.0
> Reporter: Aljoscha Krettek
> Assignee: Aljoscha Krettek
> Priority: Blocker
>
> As per the discussion on the Dev ML:
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Time-Behavior-in-Streaming-Jobs-Event-time-processing-time-td9616.html.
> The discussion seems to have converged on option 2):
> - Add dedicated WindowAssigners for processing time and event time
> - {{timeWindow()}} and {{timeWindowAll()}} respect the set
> {{TimeCharacteristic}}.
> This will make the easy stuff easy, i.e. using time windows and quickly
> switching the time characteristic. Users will then have the flexibility to
> mix different kinds of window assigners in their job.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)