1818
1919package org .apache .flink .table .runtime .operators .join .stream ;
2020
21- import org .apache .flink .api .common .functions .AbstractRichFunction ;
2221import org .apache .flink .api .java .tuple .Tuple2 ;
2322import org .apache .flink .configuration .Configuration ;
2423import org .apache .flink .streaming .api .operators .AbstractStreamOperator ;
2524import org .apache .flink .streaming .api .operators .TimestampedCollector ;
2625import org .apache .flink .streaming .api .operators .TwoInputStreamOperator ;
2726import org .apache .flink .table .data .RowData ;
28- import org .apache .flink .table .data .binary .NullAwareGetters ;
2927import org .apache .flink .table .runtime .generated .GeneratedJoinCondition ;
3028import org .apache .flink .table .runtime .generated .JoinCondition ;
31- import org .apache .flink .table .runtime .operators .join .NullAwareJoinHelper ;
29+ import org .apache .flink .table .runtime .operators .join .JoinConditionWithNullFilters ;
3230import org .apache .flink .table .runtime .operators .join .stream .state .JoinInputSideSpec ;
3331import org .apache .flink .table .runtime .operators .join .stream .state .JoinRecordStateView ;
3432import org .apache .flink .table .runtime .operators .join .stream .state .OuterJoinRecordStateView ;
@@ -60,14 +58,7 @@ public abstract class AbstractStreamingJoinOperator extends AbstractStreamOperat
6058 protected final JoinInputSideSpec leftInputSideSpec ;
6159 protected final JoinInputSideSpec rightInputSideSpec ;
6260
63- /** Should filter null keys. */
64- private final int [] nullFilterKeys ;
65-
66- /** No keys need to filter null. */
67- private final boolean nullSafe ;
68-
69- /** Filter null to all keys. */
70- private final boolean filterAllNulls ;
61+ private final boolean [] filterNullKeys ;
7162
7263 protected final long stateRetentionTime ;
7364
@@ -88,21 +79,17 @@ public AbstractStreamingJoinOperator(
8879 this .leftInputSideSpec = leftInputSideSpec ;
8980 this .rightInputSideSpec = rightInputSideSpec ;
9081 this .stateRetentionTime = stateRetentionTime ;
91- this .nullFilterKeys = NullAwareJoinHelper .getNullFilterKeys (filterNullKeys );
92- this .nullSafe = nullFilterKeys .length == 0 ;
93- this .filterAllNulls = nullFilterKeys .length == filterNullKeys .length ;
82+ this .filterNullKeys = filterNullKeys ;
9483 }
9584
9685 @ Override
9786 public void open () throws Exception {
9887 super .open ();
99-
10088 JoinCondition condition =
10189 generatedJoinCondition .newInstance (getRuntimeContext ().getUserCodeClassLoader ());
102- condition .setRuntimeContext (getRuntimeContext ());
103- condition .open (new Configuration ());
104-
105- this .joinCondition = new JoinConditionWithNullFilters (condition );
90+ this .joinCondition = new JoinConditionWithNullFilters (condition , filterNullKeys , this );
91+ this .joinCondition .setRuntimeContext (getRuntimeContext ());
92+ this .joinCondition .open (new Configuration ());
10693
10794 this .collector = new TimestampedCollector <>(output );
10895 }
@@ -111,35 +98,7 @@ public void open() throws Exception {
11198 public void close () throws Exception {
11299 super .close ();
113100 if (joinCondition != null ) {
114- joinCondition .backingJoinCondition .close ();
115- }
116- }
117-
118- // ----------------------------------------------------------------------------------------
119- // Utility Classes
120- // ----------------------------------------------------------------------------------------
121-
122- private class JoinConditionWithNullFilters extends AbstractRichFunction
123- implements JoinCondition {
124-
125- final JoinCondition backingJoinCondition ;
126-
127- private JoinConditionWithNullFilters (JoinCondition backingJoinCondition ) {
128- this .backingJoinCondition = backingJoinCondition ;
129- }
130-
131- @ Override
132- public boolean apply (RowData left , RowData right ) {
133- if (!nullSafe ) { // is not null safe, return false if any null exists
134- // key is always BinaryRowData
135- NullAwareGetters joinKey = (NullAwareGetters ) getCurrentKey ();
136- if (filterAllNulls ? joinKey .anyNull () : joinKey .anyNull (nullFilterKeys )) {
137- // find null present, return false directly
138- return false ;
139- }
140- }
141- // test condition
142- return backingJoinCondition .apply (left , right );
101+ joinCondition .close ();
143102 }
144103 }
145104
0 commit comments