19
19
package org .apache .flink .datastream .impl .operators ;
20
20
21
21
import org .apache .flink .api .java .functions .KeySelector ;
22
+ import org .apache .flink .datastream .api .context .NonPartitionedContext ;
22
23
import org .apache .flink .datastream .api .context .ProcessingTimeManager ;
23
24
import org .apache .flink .datastream .api .function .TwoInputNonBroadcastStreamProcessFunction ;
24
25
import org .apache .flink .datastream .api .stream .KeyedPartitionStream ;
25
26
import org .apache .flink .datastream .impl .common .KeyCheckedOutputCollector ;
26
27
import org .apache .flink .datastream .impl .common .OutputCollector ;
27
28
import org .apache .flink .datastream .impl .common .TimestampCollector ;
29
+ import org .apache .flink .datastream .impl .context .DefaultNonPartitionedContext ;
28
30
import org .apache .flink .datastream .impl .context .DefaultProcessingTimeManager ;
29
31
import org .apache .flink .runtime .state .VoidNamespace ;
30
32
import org .apache .flink .runtime .state .VoidNamespaceSerializer ;
31
33
import org .apache .flink .streaming .api .operators .InternalTimer ;
32
34
import org .apache .flink .streaming .api .operators .InternalTimerService ;
33
35
import org .apache .flink .streaming .api .operators .Triggerable ;
36
+ import org .apache .flink .streaming .runtime .streamrecord .StreamRecord ;
34
37
35
38
import javax .annotation .Nullable ;
36
39
40
+ import java .util .HashSet ;
41
+ import java .util .Set ;
42
+
37
43
/**
38
44
* Operator for {@link TwoInputNonBroadcastStreamProcessFunction} in {@link KeyedPartitionStream}.
39
45
*/
@@ -42,6 +48,9 @@ public class KeyedTwoInputNonBroadcastProcessOperator<KEY, IN1, IN2, OUT>
42
48
implements Triggerable <KEY , VoidNamespace > {
43
49
private transient InternalTimerService <VoidNamespace > timerService ;
44
50
51
+ // TODO Restore this keySet when task initialized from checkpoint.
52
+ private transient Set <Object > keySet ;
53
+
45
54
@ Nullable private final KeySelector <OUT , KEY > outKeySelector ;
46
55
47
56
public KeyedTwoInputNonBroadcastProcessOperator (
@@ -60,6 +69,7 @@ public KeyedTwoInputNonBroadcastProcessOperator(
60
69
public void open () throws Exception {
61
70
this .timerService =
62
71
getInternalTimerService ("processing timer" , VoidNamespaceSerializer .INSTANCE , this );
72
+ this .keySet = new HashSet <>();
63
73
super .open ();
64
74
}
65
75
@@ -98,4 +108,32 @@ public void onProcessingTime(InternalTimer<KEY, VoidNamespace> timer) throws Exc
98
108
partitionedContext ),
99
109
timer .getKey ());
100
110
}
111
+
112
+ @ Override
113
+ protected NonPartitionedContext <OUT > getNonPartitionedContext () {
114
+ return new DefaultNonPartitionedContext <>(
115
+ context , partitionedContext , collector , true , keySet );
116
+ }
117
+
118
+ @ Override
119
+ @ SuppressWarnings ({"unchecked" , "rawtypes" })
120
+ public void setKeyContextElement1 (StreamRecord record ) throws Exception {
121
+ setKeyContextElement (record , getStateKeySelector1 ());
122
+ }
123
+
124
+ @ Override
125
+ @ SuppressWarnings ({"unchecked" , "rawtypes" })
126
+ public void setKeyContextElement2 (StreamRecord record ) throws Exception {
127
+ setKeyContextElement (record , getStateKeySelector2 ());
128
+ }
129
+
130
+ private <T > void setKeyContextElement (StreamRecord <T > record , KeySelector <T , ?> selector )
131
+ throws Exception {
132
+ if (selector == null ) {
133
+ return ;
134
+ }
135
+ Object key = selector .getKey (record .getValue ());
136
+ setCurrentKey (key );
137
+ keySet .add (key );
138
+ }
101
139
}
0 commit comments