Skip to content

Commit

Permalink
[FLINK-3536] Make clearer distinction between event time and processi…
Browse files Browse the repository at this point in the history
…ng time

This brings it more in line with *ProcessingTimeWindows and makes it
clear what type of window assigner it is.

The old name, i.e. SlidingTimeWindows and TumblingTimeWindows is still
available but deprecated.
  • Loading branch information
aljoscha committed Feb 29, 2016
1 parent 0ac2b1a commit 9922d10
Show file tree
Hide file tree
Showing 31 changed files with 364 additions and 262 deletions.
16 changes: 8 additions & 8 deletions docs/apis/streaming/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,7 @@ keyedStream.maxBy("key");
key according to some characteristic (e.g., the data that arrived within the last 5 seconds).
See <a href="windows.html">windows</a> for a complete description of windows.
{% highlight java %}
dataStream.keyBy(0).window(TumblingTimeWindows.of(Time.seconds(5))); // Last 5 seconds of data
dataStream.keyBy(0).window(TumblingEventTimeWindows.of(Time.seconds(5))); // Last 5 seconds of data
{% endhighlight %}
</p>
</td>
Expand All @@ -307,7 +307,7 @@ dataStream.keyBy(0).window(TumblingTimeWindows.of(Time.seconds(5))); // Last 5 s
<p><strong>WARNING:</strong> This is in many cases a <strong>non-parallel</strong> transformation. All records will be
gathered in one task for the windowAll operator.</p>
{% highlight java %}
dataStream.windowAll(TumblingTimeWindows.of(Time.seconds(5))); // Last 5 seconds of data
dataStream.windowAll(TumblingEventTimeWindows.of(Time.seconds(5))); // Last 5 seconds of data
{% endhighlight %}
</td>
</tr>
Expand Down Expand Up @@ -410,7 +410,7 @@ dataStream.union(otherStream1, otherStream2, ...);
{% highlight java %}
dataStream.join(otherStream)
.where(0).equalTo(1)
.window(TumblingTimeWindows.of(Time.seconds(3)))
.window(TumblingEventTimeWindows.of(Time.seconds(3)))
.apply (new JoinFunction () {...});
{% endhighlight %}
</td>
Expand All @@ -422,7 +422,7 @@ dataStream.join(otherStream)
{% highlight java %}
dataStream.coGroup(otherStream)
.where(0).equalTo(1)
.window(TumblingTimeWindows.of(Time.seconds(3)))
.window(TumblingEventTimeWindows.of(Time.seconds(3)))
.apply (new CoGroupFunction () {...});
{% endhighlight %}
</td>
Expand Down Expand Up @@ -669,7 +669,7 @@ keyedStream.maxBy("key")
key according to some characteristic (e.g., the data that arrived within the last 5 seconds).
See <a href="windows.html">windows</a> for a description of windows.
{% highlight scala %}
dataStream.keyBy(0).window(TumblingTimeWindows.of(Time.seconds(5))) // Last 5 seconds of data
dataStream.keyBy(0).window(TumblingEventTimeWindows.of(Time.seconds(5))) // Last 5 seconds of data
{% endhighlight %}
</p>
</td>
Expand All @@ -683,7 +683,7 @@ dataStream.keyBy(0).window(TumblingTimeWindows.of(Time.seconds(5))) // Last 5 se
<p><strong>WARNING:</strong> This is in many cases a <strong>non-parallel</strong> transformation. All records will be
gathered in one task for the windowAll operator.</p>
{% highlight scala %}
dataStream.windowAll(TumblingTimeWindows.of(Time.seconds(5))) // Last 5 seconds of data
dataStream.windowAll(TumblingEventTimeWindows.of(Time.seconds(5))) // Last 5 seconds of data
{% endhighlight %}
</td>
</tr>
Expand Down Expand Up @@ -759,7 +759,7 @@ dataStream.union(otherStream1, otherStream2, ...)
{% highlight scala %}
dataStream.join(otherStream)
.where(0).equalTo(1)
.window(TumblingTimeWindows.of(Time.seconds(3)))
.window(TumblingEventTimeWindows.of(Time.seconds(3)))
.apply { ... }
{% endhighlight %}
</td>
Expand All @@ -771,7 +771,7 @@ dataStream.join(otherStream)
{% highlight scala %}
dataStream.coGroup(otherStream)
.where(0).equalTo(1)
.window(TumblingTimeWindows.of(Time.seconds(3)))
.window(TumblingEventTimeWindows.of(Time.seconds(3)))
.apply {}
{% endhighlight %}
</td>
Expand Down
2 changes: 1 addition & 1 deletion docs/apis/streaming/time.md
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ windowing you would use window assigners such as `SlidingProcessingTimeWindows`
`TumblingProcessingTimeWindows`.

In order to work with event time semantics, i.e. if you want to use window assigners such as
`TumblingTimeWindows` or `SlidingTimeWindows`, you need to follow these steps:
`TumblingEventTimeWindows` or `SlidingEventTimeWindows`, you need to follow these steps:

- Set `enableTimestamps()`, as well the interval for watermark emission
(`setAutoWatermarkInterval(long milliseconds)`) in `ExecutionConfig`.
Expand Down
31 changes: 15 additions & 16 deletions docs/apis/streaming/windows.md
Original file line number Diff line number Diff line change
Expand Up @@ -191,7 +191,7 @@ window, and every time execution is triggered, 10 elements are retained in the w
<div data-lang="java" markdown="1">
{% highlight java %}
keyedStream
.window(SlidingTimeWindows.of(Time.seconds(5), Time.seconds(1))
.window(SlidingEventTimeWindows.of(Time.seconds(5), Time.seconds(1))
.trigger(CountTrigger.of(100))
.evictor(CountEvictor.of(10));
{% endhighlight %}
Expand All @@ -200,7 +200,7 @@ keyedStream
<div data-lang="scala" markdown="1">
{% highlight scala %}
keyedStream
.window(SlidingTimeWindows.of(Time.seconds(5), Time.seconds(1))
.window(SlidingEventTimeWindows.of(Time.seconds(5), Time.seconds(1))
.trigger(CountTrigger.of(100))
.evictor(CountEvictor.of(10))
{% endhighlight %}
Expand All @@ -214,7 +214,7 @@ The `WindowAssigner` defines how incoming elements are assigned to windows. A wi
that has a begin-value, and an end-value corresponding to a begin-time and end-time. Elements with timestamp (according
to some notion of time described above within these values are part of the window).

For example, the `SlidingTimeWindows`
For example, the `SlidingEventTimeWindows`
assigner in the code above defines a window of size 5 seconds, and a slide of 1 second. Assume that
time starts from 0 and is measured in milliseconds. Then, we have 6 windows
that overlap: [0,5000], [1000,6000], [2000,7000], [3000, 8000], [4000, 9000], and [5000, 10000]. Each incoming
Expand Down Expand Up @@ -256,7 +256,7 @@ stream.window(GlobalWindows.create());
watermark with value higher than its end-value is received.
</p>
{% highlight java %}
stream.window(TumblingTimeWindows.of(Time.seconds(1)));
stream.window(TumblingEventTimeWindows.of(Time.seconds(1)));
{% endhighlight %}
</td>
</tr>
Expand All @@ -270,7 +270,7 @@ stream.window(TumblingTimeWindows.of(Time.seconds(1)));
watermark with value higher than its end-value is received.
</p>
{% highlight java %}
stream.window(SlidingTimeWindows.of(Time.seconds(5), Time.seconds(1)));
stream.window(SlidingEventTimeWindows.of(Time.seconds(5), Time.seconds(1)));
{% endhighlight %}
</td>
</tr>
Expand Down Expand Up @@ -338,7 +338,7 @@ stream.window(GlobalWindows.create)
watermark with value higher than its end-value is received.
</p>
{% highlight scala %}
stream.window(TumblingTimeWindows.of(Time.seconds(1)))
stream.window(TumblingEventTimeWindows.of(Time.seconds(1)))
{% endhighlight %}
</td>
</tr>
Expand All @@ -352,7 +352,7 @@ stream.window(TumblingTimeWindows.of(Time.seconds(1)))
watermark with value higher than its end-value is received.
</p>
{% highlight scala %}
stream.window(SlidingTimeWindows.of(Time.seconds(5), Time.seconds(1)))
stream.window(SlidingEventTimeWindows.of(Time.seconds(5), Time.seconds(1)))
{% endhighlight %}
</td>
</tr>
Expand Down Expand Up @@ -743,8 +743,7 @@ stream.countWindow(1000)
<td>
{% highlight java %}
stream.window(GlobalWindows.create())
.trigger(CountTrigger.of(1000)
.evictor(CountEvictor.of(1000)))
.trigger(PurgingTrigger.of(CountTrigger.of(size)))
{% endhighlight %}
</td>
</tr>
Expand Down Expand Up @@ -772,7 +771,7 @@ stream.timeWindow(Time.seconds(5))
</td>
<td>
{% highlight java %}
stream.window(TumblingTimeWindows.of((Time.seconds(5)))
stream.window(TumblingEventTimeWindows.of((Time.seconds(5)))
.trigger(EventTimeTrigger.create())
{% endhighlight %}
</td>
Expand All @@ -786,7 +785,7 @@ stream.timeWindow(Time.seconds(5), Time.seconds(1))
</td>
<td>
{% highlight java %}
stream.window(SlidingTimeWindows.of(Time.seconds(5), Time.seconds(1)))
stream.window(SlidingEventTimeWindows.of(Time.seconds(5), Time.seconds(1)))
.trigger(EventTimeTrigger.create())
{% endhighlight %}
</td>
Expand All @@ -800,7 +799,7 @@ stream.timeWindow(Time.seconds(5))
</td>
<td>
{% highlight java %}
stream.window(TumblingTimeWindows.of((Time.seconds(5)))
stream.window(TumblingProcessingTimeWindows.of((Time.seconds(5)))
.trigger(ProcessingTimeTrigger.create())
{% endhighlight %}
</td>
Expand All @@ -814,7 +813,7 @@ stream.timeWindow(Time.seconds(5), Time.seconds(1))
</td>
<td>
{% highlight java %}
stream.window(SlidingTimeWindows.of(Time.seconds(5), Time.seconds(1)))
stream.window(SlidingProcessingTimeWindows.of(Time.seconds(5), Time.seconds(1)))
.trigger(ProcessingTimeTrigger.create())
{% endhighlight %}
</td>
Expand All @@ -834,7 +833,7 @@ same:
<div data-lang="java" markdown="1">
{% highlight java %}
nonKeyedStream
.windowAll(SlidingTimeWindows.of(Time.seconds(5), Time.seconds(1))
.windowAll(SlidingEventTimeWindows.of(Time.seconds(5), Time.seconds(1))
.trigger(CountTrigger.of(100))
.evictor(CountEvictor.of(10));
{% endhighlight %}
Expand All @@ -843,7 +842,7 @@ nonKeyedStream
<div data-lang="scala" markdown="1">
{% highlight scala %}
nonKeyedStream
.windowAll(SlidingTimeWindows.of(Time.seconds(5), Time.seconds(1))
.windowAll(SlidingEventTimeWindows.of(Time.seconds(5), Time.seconds(1))
.trigger(CountTrigger.of(100))
.evictor(CountEvictor.of(10))
{% endhighlight %}
Expand Down Expand Up @@ -992,4 +991,4 @@ nonKeyedStream.countWindowAll(1000, 100)
</table>

</div>
</div>
</div>
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
import org.apache.flink.streaming.api.functions.AscendingTimestampExtractor;
import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
import org.apache.flink.streaming.api.windowing.time.Time;

import java.util.Random;
Expand Down Expand Up @@ -84,7 +84,7 @@ public static void main(String[] args) throws Exception {
.join(salaries)
.where(new NameKeySelector())
.equalTo(new NameKeySelector())
.window(TumblingTimeWindows.of(Time.of(5, TimeUnit.MILLISECONDS)))
.window(TumblingEventTimeWindows.of(Time.of(5, TimeUnit.MILLISECONDS)))
.apply(new MyJoinFunction());

// emit result
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ package org.apache.flink.streaming.scala.examples.join
import org.apache.flink.api.java.utils.ParameterTool
import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.scala._
import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows
import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows
import org.apache.flink.streaming.api.windowing.time.Time

import scala.Stream._
Expand Down Expand Up @@ -56,7 +56,7 @@ object WindowJoin {
val joined = grades.join(salaries)
.where(_.name)
.equalTo(_.name)
.window(SlidingTimeWindows.of(Time.seconds(2), Time.seconds(1)))
.window(SlidingEventTimeWindows.of(Time.seconds(2), Time.seconds(1)))
.apply { (g, s) => Person(g.name, g.grade, s.salary) }

if (params.has("output")) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@
* DataStream<T> result = one.coGroup(two)
* .where(new MyFirstKeySelector())
* .equalTo(new MyFirstKeySelector())
* .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
* .window(TumblingEventTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
* .apply(new MyCoGroupFunction());
* } </pre>
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,9 +66,9 @@
import org.apache.flink.streaming.api.transformations.UnionTransformation;
import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
import org.apache.flink.streaming.api.windowing.time.Time;
Expand Down Expand Up @@ -594,7 +594,7 @@ public <T2> JoinedStreams<T, T2> join(DataStream<T2> otherStream) {
* Windows this {@code DataStream} into tumbling time windows.
*
* <p>
* This is a shortcut for either {@code .window(TumblingTimeWindows.of(size))} or
* This is a shortcut for either {@code .window(TumblingEventTimeWindows.of(size))} or
* {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic
* set using
*
Expand All @@ -611,15 +611,15 @@ public AllWindowedStream<T, TimeWindow> timeWindowAll(Time size) {
if (environment.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
return windowAll(TumblingProcessingTimeWindows.of(size));
} else {
return windowAll(TumblingTimeWindows.of(size));
return windowAll(TumblingEventTimeWindows.of(size));
}
}

/**
* Windows this {@code DataStream} into sliding time windows.
*
* <p>
* This is a shortcut for either {@code .window(SlidingTimeWindows.of(size, slide))} or
* This is a shortcut for either {@code .window(SlidingEventTimeWindows.of(size, slide))} or
* {@code .window(SlidingProcessingTimeWindows.of(size, slide))} depending on the time characteristic
* set using
* {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
Expand All @@ -635,7 +635,7 @@ public AllWindowedStream<T, TimeWindow> timeWindowAll(Time size, Time slide) {
if (environment.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
return windowAll(SlidingProcessingTimeWindows.of(size, slide));
} else {
return windowAll(SlidingTimeWindows.of(size, slide));
return windowAll(SlidingEventTimeWindows.of(size, slide));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@
* DataStream<T> result = one.join(two)
* .where(new MyFirstKeySelector())
* .equalTo(new MyFirstKeySelector())
* .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
* .window(TumblingEventTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
* .apply(new MyJoinFunction());
* } </pre>
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,9 @@
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
import org.apache.flink.streaming.api.windowing.time.Time;
Expand Down Expand Up @@ -165,7 +165,7 @@ public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
* Windows this {@code KeyedStream} into tumbling time windows.
*
* <p>
* This is a shortcut for either {@code .window(TumblingTimeWindows.of(size))} or
* This is a shortcut for either {@code .window(TumblingEventTimeWindows.of(size))} or
* {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic
* set using
* {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
Expand All @@ -176,15 +176,15 @@ public WindowedStream<T, KEY, TimeWindow> timeWindow(Time size) {
if (environment.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
return window(TumblingProcessingTimeWindows.of(size));
} else {
return window(TumblingTimeWindows.of(size));
return window(TumblingEventTimeWindows.of(size));
}
}

/**
* Windows this {@code KeyedStream} into sliding time windows.
*
* <p>
* This is a shortcut for either {@code .window(SlidingTimeWindows.of(size, slide))} or
* This is a shortcut for either {@code .window(SlidingEventTimeWindows.of(size, slide))} or
* {@code .window(SlidingProcessingTimeWindows.of(size, slide))} depending on the time characteristic
* set using
* {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
Expand All @@ -195,7 +195,7 @@ public WindowedStream<T, KEY, TimeWindow> timeWindow(Time size, Time slide) {
if (environment.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
return window(SlidingProcessingTimeWindows.of(size, slide));
} else {
return window(SlidingTimeWindows.of(size, slide));
return window(SlidingEventTimeWindows.of(size, slide));
}
}

Expand Down
Loading

0 comments on commit 9922d10

Please sign in to comment.