@@ -355,12 +355,8 @@ public WithWindow<T1, T2, KEY, W> allowedLateness(@Nullable Duration newLateness
355
355
/**
356
356
* Completes the join operation with the user function that is executed for each combination
357
357
* 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.
362
358
*/
363
- public <T > DataStream <T > apply (JoinFunction <T1 , T2 , T > function ) {
359
+ public <T > SingleOutputStreamOperator <T > apply (JoinFunction <T1 , T2 , T > function ) {
364
360
TypeInformation <T > resultType =
365
361
TypeExtractor .getBinaryOperatorReturnType (
366
362
function ,
@@ -380,30 +376,8 @@ public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function) {
380
376
/**
381
377
* Completes the join operation with the user function that is executed for each combination
382
378
* 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.
405
379
*/
406
- public <T > DataStream <T > apply (
380
+ public <T > SingleOutputStreamOperator <T > apply (
407
381
FlatJoinFunction <T1 , T2 , T > function , TypeInformation <T > resultType ) {
408
382
// clean the closure
409
383
function = input1 .getExecutionEnvironment ().clean (function );
@@ -424,31 +398,8 @@ public <T> DataStream<T> apply(
424
398
/**
425
399
* Completes the join operation with the user function that is executed for each combination
426
400
* 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.
450
401
*/
451
- public <T > DataStream <T > apply (FlatJoinFunction <T1 , T2 , T > function ) {
402
+ public <T > SingleOutputStreamOperator <T > apply (FlatJoinFunction <T1 , T2 , T > function ) {
452
403
TypeInformation <T > resultType =
453
404
TypeExtractor .getBinaryOperatorReturnType (
454
405
function ,
@@ -468,30 +419,8 @@ public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function) {
468
419
/**
469
420
* Completes the join operation with the user function that is executed for each combination
470
421
* 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.
493
422
*/
494
- public <T > DataStream <T > apply (
423
+ public <T > SingleOutputStreamOperator <T > apply (
495
424
JoinFunction <T1 , T2 , T > function , TypeInformation <T > resultType ) {
496
425
// clean the closure
497
426
function = input1 .getExecutionEnvironment ().clean (function );
@@ -508,24 +437,6 @@ public <T> DataStream<T> apply(
508
437
return coGroupedWindowedStream .apply (new JoinCoGroupFunction <>(function ), resultType );
509
438
}
510
439
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
-
529
440
/** @deprecated Use {@link #getAllowedLatenessDuration()}} */
530
441
@ VisibleForTesting
531
442
@ Nullable
0 commit comments