Skip to content

Commit be4549d

Browse files
yunfengzhou-hubreswqa
authored andcommitted
[FLINK-36355][runtime] Remove deprecated xxxStreams#with
1 parent 21092bf commit be4549d

File tree

2 files changed

+6
-139
lines changed

2 files changed

+6
-139
lines changed

flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java

Lines changed: 2 additions & 46 deletions
Original file line numberDiff line numberDiff line change
@@ -364,12 +364,8 @@ public WithWindow<T1, T2, KEY, W> allowedLateness(@Nullable Duration newLateness
364364
/**
365365
* Completes the co-group operation with the user function that is executed for windowed
366366
* groups.
367-
*
368-
* <p>Note: This method's return type does not support setting an operator-specific
369-
* parallelism. Due to binary backwards compatibility, this cannot be altered. Use the
370-
* {@link #with(CoGroupFunction)} method to set an operator-specific parallelism.
371367
*/
372-
public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function) {
368+
public <T> SingleOutputStreamOperator<T> apply(CoGroupFunction<T1, T2, T> function) {
373369

374370
TypeInformation<T> resultType =
375371
TypeExtractor.getCoGroupReturnTypes(
@@ -381,30 +377,8 @@ public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function) {
381377
/**
382378
* Completes the co-group operation with the user function that is executed for windowed
383379
* groups.
384-
*
385-
* <p><b>Note:</b> This is a temporary workaround while the {@link #apply(CoGroupFunction)}
386-
* method has the wrong return type and hence does not allow one to set an operator-specific
387-
* parallelism
388-
*
389-
* @deprecated This method will be removed once the {@link #apply(CoGroupFunction)} method
390-
* is fixed in the next major version of Flink (2.0).
391-
*/
392-
@PublicEvolving
393-
@Deprecated
394-
public <T> SingleOutputStreamOperator<T> with(CoGroupFunction<T1, T2, T> function) {
395-
return (SingleOutputStreamOperator<T>) apply(function);
396-
}
397-
398-
/**
399-
* Completes the co-group operation with the user function that is executed for windowed
400-
* groups.
401-
*
402-
* <p>Note: This method's return type does not support setting an operator-specific
403-
* parallelism. Due to binary backwards compatibility, this cannot be altered. Use the
404-
* {@link #with(CoGroupFunction, TypeInformation)} method to set an operator-specific
405-
* parallelism.
406380
*/
407-
public <T> DataStream<T> apply(
381+
public <T> SingleOutputStreamOperator<T> apply(
408382
CoGroupFunction<T1, T2, T> function, TypeInformation<T> resultType) {
409383
// clean the closure
410384
function = input1.getExecutionEnvironment().clean(function);
@@ -446,24 +420,6 @@ public <T> DataStream<T> apply(
446420
new CoGroupWindowFunction<T1, T2, T, KEY, W>(function), resultType);
447421
}
448422

449-
/**
450-
* Completes the co-group operation with the user function that is executed for windowed
451-
* groups.
452-
*
453-
* <p><b>Note:</b> This is a temporary workaround while the {@link #apply(CoGroupFunction,
454-
* TypeInformation)} method has the wrong return type and hence does not allow one to set an
455-
* operator-specific parallelism
456-
*
457-
* @deprecated This method will be removed once the {@link #apply(CoGroupFunction,
458-
* TypeInformation)} method is fixed in the next major version of Flink (2.0).
459-
*/
460-
@PublicEvolving
461-
@Deprecated
462-
public <T> SingleOutputStreamOperator<T> with(
463-
CoGroupFunction<T1, T2, T> function, TypeInformation<T> resultType) {
464-
return (SingleOutputStreamOperator<T>) apply(function, resultType);
465-
}
466-
467423
/** @deprecated Use {@link #getAllowedLatenessDuration()} */
468424
@Deprecated
469425
@VisibleForTesting

flink-runtime/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java

Lines changed: 4 additions & 93 deletions
Original file line numberDiff line numberDiff line change
@@ -355,12 +355,8 @@ public WithWindow<T1, T2, KEY, W> allowedLateness(@Nullable Duration newLateness
355355
/**
356356
* Completes the join operation with the user function that is executed for each combination
357357
* of elements with the same key in a window.
358-
*
359-
* <p>Note: This method's return type does not support setting an operator-specific
360-
* parallelism. Due to binary backwards compatibility, this cannot be altered. Use the
361-
* {@link #with(JoinFunction)} method to set an operator-specific parallelism.
362358
*/
363-
public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function) {
359+
public <T> SingleOutputStreamOperator<T> apply(JoinFunction<T1, T2, T> function) {
364360
TypeInformation<T> resultType =
365361
TypeExtractor.getBinaryOperatorReturnType(
366362
function,
@@ -380,30 +376,8 @@ public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function) {
380376
/**
381377
* Completes the join operation with the user function that is executed for each combination
382378
* of elements with the same key in a window.
383-
*
384-
* <p><b>Note:</b> This is a temporary workaround while the {@link #apply(JoinFunction)}
385-
* method has the wrong return type and hence does not allow one to set an operator-specific
386-
* parallelism
387-
*
388-
* @deprecated This method will be removed once the {@link #apply(JoinFunction)} method is
389-
* fixed in the next major version of Flink (2.0).
390-
*/
391-
@PublicEvolving
392-
@Deprecated
393-
public <T> SingleOutputStreamOperator<T> with(JoinFunction<T1, T2, T> function) {
394-
return (SingleOutputStreamOperator<T>) apply(function);
395-
}
396-
397-
/**
398-
* Completes the join operation with the user function that is executed for each combination
399-
* of elements with the same key in a window.
400-
*
401-
* <p>Note: This method's return type does not support setting an operator-specific
402-
* parallelism. Due to binary backwards compatibility, this cannot be altered. Use the
403-
* {@link #with(JoinFunction, TypeInformation)}, method to set an operator-specific
404-
* parallelism.
405379
*/
406-
public <T> DataStream<T> apply(
380+
public <T> SingleOutputStreamOperator<T> apply(
407381
FlatJoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
408382
// clean the closure
409383
function = input1.getExecutionEnvironment().clean(function);
@@ -424,31 +398,8 @@ public <T> DataStream<T> apply(
424398
/**
425399
* Completes the join operation with the user function that is executed for each combination
426400
* of elements with the same key in a window.
427-
*
428-
* <p><b>Note:</b> This is a temporary workaround while the {@link #apply(JoinFunction,
429-
* TypeInformation)} method has the wrong return type and hence does not allow one to set an
430-
* operator-specific parallelism
431-
*
432-
* @deprecated This method will be replaced by {@link #apply(FlatJoinFunction,
433-
* TypeInformation)} in Flink 2.0. So use the {@link #apply(FlatJoinFunction,
434-
* TypeInformation)} in the future.
435-
*/
436-
@PublicEvolving
437-
@Deprecated
438-
public <T> SingleOutputStreamOperator<T> with(
439-
FlatJoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
440-
return (SingleOutputStreamOperator<T>) apply(function, resultType);
441-
}
442-
443-
/**
444-
* Completes the join operation with the user function that is executed for each combination
445-
* of elements with the same key in a window.
446-
*
447-
* <p>Note: This method's return type does not support setting an operator-specific
448-
* parallelism. Due to binary backwards compatibility, this cannot be altered. Use the
449-
* {@link #with(FlatJoinFunction)}, method to set an operator-specific parallelism.
450401
*/
451-
public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function) {
402+
public <T> SingleOutputStreamOperator<T> apply(FlatJoinFunction<T1, T2, T> function) {
452403
TypeInformation<T> resultType =
453404
TypeExtractor.getBinaryOperatorReturnType(
454405
function,
@@ -468,30 +419,8 @@ public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function) {
468419
/**
469420
* Completes the join operation with the user function that is executed for each combination
470421
* of elements with the same key in a window.
471-
*
472-
* <p><b>Note:</b> This is a temporary workaround while the {@link #apply(FlatJoinFunction)}
473-
* method has the wrong return type and hence does not allow one to set an operator-specific
474-
* parallelism.
475-
*
476-
* @deprecated This method will be removed once the {@link #apply(FlatJoinFunction)} method
477-
* is fixed in the next major version of Flink (2.0).
478-
*/
479-
@PublicEvolving
480-
@Deprecated
481-
public <T> SingleOutputStreamOperator<T> with(FlatJoinFunction<T1, T2, T> function) {
482-
return (SingleOutputStreamOperator<T>) apply(function);
483-
}
484-
485-
/**
486-
* Completes the join operation with the user function that is executed for each combination
487-
* of elements with the same key in a window.
488-
*
489-
* <p>Note: This method's return type does not support setting an operator-specific
490-
* parallelism. Due to binary backwards compatibility, this cannot be altered. Use the
491-
* {@link #with(JoinFunction, TypeInformation)}, method to set an operator-specific
492-
* parallelism.
493422
*/
494-
public <T> DataStream<T> apply(
423+
public <T> SingleOutputStreamOperator<T> apply(
495424
JoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
496425
// clean the closure
497426
function = input1.getExecutionEnvironment().clean(function);
@@ -508,24 +437,6 @@ public <T> DataStream<T> apply(
508437
return coGroupedWindowedStream.apply(new JoinCoGroupFunction<>(function), resultType);
509438
}
510439

511-
/**
512-
* Completes the join operation with the user function that is executed for each combination
513-
* of elements with the same key in a window.
514-
*
515-
* <p><b>Note:</b> This is a temporary workaround while the {@link #apply(FlatJoinFunction,
516-
* TypeInformation)} method has the wrong return type and hence does not allow one to set an
517-
* operator-specific parallelism
518-
*
519-
* @deprecated This method will be removed once the {@link #apply(JoinFunction,
520-
* TypeInformation)} method is fixed in the next major version of Flink (2.0).
521-
*/
522-
@PublicEvolving
523-
@Deprecated
524-
public <T> SingleOutputStreamOperator<T> with(
525-
JoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
526-
return (SingleOutputStreamOperator<T>) apply(function, resultType);
527-
}
528-
529440
/** @deprecated Use {@link #getAllowedLatenessDuration()}} */
530441
@VisibleForTesting
531442
@Nullable

0 commit comments

Comments
 (0)