@@ -30,10 +30,11 @@ import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction
30
30
import org .apache .flink .streaming .api .functions .query .{QueryableAppendingStateOperator , QueryableValueStateOperator }
31
31
import org .apache .flink .streaming .api .scala .function .StatefulFunction
32
32
import org .apache .flink .streaming .api .windowing .assigners ._
33
- import org .apache .flink .streaming .api .windowing .time .Time
34
- import org .apache .flink .streaming .api .windowing .windows .{GlobalWindow , TimeWindow , Window }
33
+ import org .apache .flink .streaming .api .windowing .windows .{GlobalWindow , Window }
35
34
import org .apache .flink .util .Collector
36
35
36
+ import java .time .Duration
37
+
37
38
/**
38
39
* @deprecated
39
40
* All Flink Scala APIs are deprecated and will be removed in a future Flink major version. You
@@ -162,9 +163,9 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
162
163
* The upper bound. Needs to be bigger than or equal to the lowerBound
163
164
*/
164
165
@ PublicEvolving
165
- def between (lowerBound : Time , upperBound : Time ): IntervalJoined [IN1 , IN2 , KEY ] = {
166
- val lowerMillis = lowerBound.toMilliseconds
167
- val upperMillis = upperBound.toMilliseconds
166
+ def between (lowerBound : Duration , upperBound : Duration ): IntervalJoined [IN1 , IN2 , KEY ] = {
167
+ val lowerMillis = lowerBound.toMillis
168
+ val upperMillis = upperBound.toMillis
168
169
new IntervalJoined [IN1 , IN2 , KEY ](streamOne, streamTwo, lowerMillis, upperMillis)
169
170
}
170
171
}
@@ -236,46 +237,6 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
236
237
// Windowing
237
238
// ------------------------------------------------------------------------
238
239
239
- /**
240
- * Windows this [[KeyedStream ]] into tumbling time windows.
241
- *
242
- * This is a shortcut for either `.window(TumblingEventTimeWindows.of(size))` or
243
- * `.window(TumblingProcessingTimeWindows.of(size))` depending on the time characteristic set
244
- * using [[StreamExecutionEnvironment.setStreamTimeCharacteristic() ]]
245
- *
246
- * @param size
247
- * The size of the window.
248
- *
249
- * @deprecated
250
- * Please use [[window() ]] with either [[TumblingEventTimeWindows ]] or
251
- * [[TumblingProcessingTimeWindows ]]. For more information, see the deprecation notice on
252
- * [[org.apache.flink.streaming.api.TimeCharacteristic ]].
253
- */
254
- @ deprecated
255
- def timeWindow (size : Time ): WindowedStream [T , K , TimeWindow ] = {
256
- new WindowedStream (javaStream.timeWindow(size))
257
- }
258
-
259
- /**
260
- * Windows this [[KeyedStream ]] into sliding time windows.
261
- *
262
- * This is a shortcut for either `.window(SlidingEventTimeWindows.of(size))` or
263
- * `.window(SlidingProcessingTimeWindows.of(size))` depending on the time characteristic set using
264
- * [[StreamExecutionEnvironment.setStreamTimeCharacteristic() ]]
265
- *
266
- * @param size
267
- * The size of the window.
268
- *
269
- * @deprecated
270
- * Please use [[window() ]] with either [[SlidingEventTimeWindows ]] or
271
- * [[SlidingProcessingTimeWindows ]]. For more information, see the deprecation notice on
272
- * [[org.apache.flink.streaming.api.TimeCharacteristic ]].
273
- */
274
- @ deprecated
275
- def timeWindow (size : Time , slide : Time ): WindowedStream [T , K , TimeWindow ] = {
276
- new WindowedStream (javaStream.timeWindow(size, slide))
277
- }
278
-
279
240
/**
280
241
* Windows this [[KeyedStream ]] into sliding count windows.
281
242
*
0 commit comments