[FLINK-17316][docs-training-exercises] avoid using windows that depend on the time characteristic
diff --git a/hourly-tips/DISCUSSION.md b/hourly-tips/DISCUSSION.md
index ab653d3..e498f81 100644
--- a/hourly-tips/DISCUSSION.md
+++ b/hourly-tips/DISCUSSION.md
@@ -26,7 +26,7 @@
```java
DataStream<Tuple3<Long, Long, Float>> hourlyTips = fares
.keyBy((TaxiFare fare) -> fare.driverId)
- .timeWindow(Time.hours(1))
+ .window(TumblingEventTimeWindows.of(Time.hours(1)))
.process(new AddTips());
```
@@ -54,7 +54,7 @@
val hourlyTips = fares
.map((f: TaxiFare) => (f.driverId, f.tip))
.keyBy(_._1)
- .timeWindow(Time.hours(1))
+ .window(TumblingEventTimeWindows.of(Time.hours(1)))
.reduce(
(f1: (Long, Float), f2: (Long, Float)) => { (f1._1, f1._2 + f2._2) },
new WrapWithWindowInfo())
@@ -100,7 +100,7 @@
```java
DataStream<Tuple3<Long, Long, Float>> hourlyMax = hourlyTips
- .timeWindowAll(Time.hours(1))
+ .windowAll(TumblingEventTimeWindows.of(Time.hours(1)))
.maxBy(2);
```
diff --git a/hourly-tips/src/solution/java/org/apache/flink/training/solutions/hourlytips/HourlyTipsSolution.java b/hourly-tips/src/solution/java/org/apache/flink/training/solutions/hourlytips/HourlyTipsSolution.java
index b640a19..54671f9 100644
--- a/hourly-tips/src/solution/java/org/apache/flink/training/solutions/hourlytips/HourlyTipsSolution.java
+++ b/hourly-tips/src/solution/java/org/apache/flink/training/solutions/hourlytips/HourlyTipsSolution.java
@@ -24,6 +24,7 @@
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
import org.apache.flink.streaming.api.windowing.time.Time;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.training.exercises.common.datatypes.TaxiFare;
@@ -70,15 +71,15 @@
// compute tips per hour for each driver
DataStream<Tuple3<Long, Long, Float>> hourlyTips = fares
.keyBy((TaxiFare fare) -> fare.driverId)
- .timeWindow(Time.hours(1))
+ .window(TumblingEventTimeWindows.of(Time.hours(1)))
.process(new AddTips());
DataStream<Tuple3<Long, Long, Float>> hourlyMax = hourlyTips
- .timeWindowAll(Time.hours(1))
+ .windowAll(TumblingEventTimeWindows.of(Time.hours(1)))
.maxBy(2);
// You should explore how this alternative behaves. In what ways is the same as,
-// and different from, the solution above (using a timeWindowAll)?
+// and different from, the solution above (using a windowAll)?
// DataStream<Tuple3<Long, Long, Float>> hourlyMax = hourlyTips
// .keyBy(0)
diff --git a/hourly-tips/src/solution/scala/org/apache/flink/training/solutions/hourlytips/scala/HourlyTipsSolution.scala b/hourly-tips/src/solution/scala/org/apache/flink/training/solutions/hourlytips/scala/HourlyTipsSolution.scala
index ab3de6c..06db503 100644
--- a/hourly-tips/src/solution/scala/org/apache/flink/training/solutions/hourlytips/scala/HourlyTipsSolution.scala
+++ b/hourly-tips/src/solution/scala/org/apache/flink/training/solutions/hourlytips/scala/HourlyTipsSolution.scala
@@ -22,6 +22,7 @@
import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.scala._
import org.apache.flink.streaming.api.scala.function.ProcessWindowFunction
+import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows
import org.apache.flink.streaming.api.windowing.time.Time
import org.apache.flink.streaming.api.windowing.windows.TimeWindow
import org.apache.flink.training.exercises.common.datatypes.TaxiFare
@@ -62,14 +63,14 @@
val hourlyTips = fares
.map((f: TaxiFare) => (f.driverId, f.tip))
.keyBy(_._1)
- .timeWindow(Time.hours(1))
+ .window(TumblingEventTimeWindows.of(Time.hours(1)))
.reduce(
(f1: (Long, Float), f2: (Long, Float)) => { (f1._1, f1._2 + f2._2) },
new WrapWithWindowInfo())
// max tip total in each hour
val hourlyMax = hourlyTips
- .timeWindowAll(Time.hours(1))
+ .windowAll(TumblingEventTimeWindows.of(Time.hours(1)))
.maxBy(2)
// print result on stdout