7979import org .slf4j .Logger ;
8080import org .slf4j .LoggerFactory ;
8181
82+ import javax .annotation .Nullable ;
83+
8284import java .util .Arrays ;
8385import java .util .List ;
8486import java .util .Map ;
@@ -116,6 +118,9 @@ public class StreamExecDeltaJoin extends ExecNodeBase<RowData>
116118 "lookupRightTableJoinSpec" ;
117119 private static final String FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC = "lookupLeftTableJoinSpec" ;
118120
121+ private static final String FIELD_NAME_LEFT_UPSERT_KEY = "leftUpsertKey" ;
122+ private static final String FIELD_NAME_RIGHT_UPSERT_KEY = "rightUpsertKey" ;
123+
119124 private static final String FIELD_NAME_JOIN_TYPE = "joinType" ;
120125
121126 public static final String FIELD_NAME_ASYNC_OPTIONS = "asyncOptions" ;
@@ -134,6 +139,10 @@ public class StreamExecDeltaJoin extends ExecNodeBase<RowData>
134139 @ JsonProperty (FIELD_NAME_LEFT_JOIN_KEYS )
135140 private final int [] leftJoinKeys ;
136141
142+ @ JsonProperty (FIELD_NAME_LEFT_UPSERT_KEY )
143+ @ JsonInclude (JsonInclude .Include .NON_NULL )
144+ private final int [] leftUpsertKeys ;
145+
137146 // left (streaming) side join right (lookup) side
138147 @ JsonProperty (FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC )
139148 private final DeltaJoinSpec lookupRightTableJoinSpec ;
@@ -143,6 +152,10 @@ public class StreamExecDeltaJoin extends ExecNodeBase<RowData>
143152 @ JsonProperty (FIELD_NAME_RIGHT_JOIN_KEYS )
144153 private final int [] rightJoinKeys ;
145154
155+ @ JsonProperty (FIELD_NAME_RIGHT_UPSERT_KEY )
156+ @ JsonInclude (JsonInclude .Include .NON_NULL )
157+ private final int [] rightUpsertKeys ;
158+
146159 // right (streaming) side join left (lookup) side
147160 @ JsonProperty (FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC )
148161 private final DeltaJoinSpec lookupLeftTableJoinSpec ;
@@ -152,9 +165,11 @@ public StreamExecDeltaJoin(
152165 FlinkJoinType flinkJoinType ,
153166 // delta join args related with the left side
154167 int [] leftJoinKeys ,
168+ @ Nullable int [] leftUpsertKeys ,
155169 DeltaJoinSpec lookupRightTableJoinSpec ,
156170 // delta join args related with the right side
157171 int [] rightJoinKeys ,
172+ @ Nullable int [] rightUpsertKeys ,
158173 DeltaJoinSpec lookupLeftTableJoinSpec ,
159174 InputProperty leftInputProperty ,
160175 InputProperty rightInputProperty ,
@@ -167,8 +182,10 @@ public StreamExecDeltaJoin(
167182 ExecNodeContext .newPersistedConfig (StreamExecDeltaJoin .class , tableConfig ),
168183 flinkJoinType ,
169184 leftJoinKeys ,
185+ leftUpsertKeys ,
170186 lookupRightTableJoinSpec ,
171187 rightJoinKeys ,
188+ rightUpsertKeys ,
172189 lookupLeftTableJoinSpec ,
173190 Lists .newArrayList (leftInputProperty , rightInputProperty ),
174191 outputType ,
@@ -183,9 +200,11 @@ public StreamExecDeltaJoin(
183200 @ JsonProperty (FIELD_NAME_CONFIGURATION ) ReadableConfig persistedConfig ,
184201 @ JsonProperty (FIELD_NAME_JOIN_TYPE ) FlinkJoinType flinkJoinType ,
185202 @ JsonProperty (FIELD_NAME_LEFT_JOIN_KEYS ) int [] leftJoinKeys ,
203+ @ JsonProperty (FIELD_NAME_LEFT_UPSERT_KEY ) @ Nullable int [] leftUpsertKeys ,
186204 @ JsonProperty (FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC )
187205 DeltaJoinSpec lookupRightTableJoinSpec ,
188206 @ JsonProperty (FIELD_NAME_RIGHT_JOIN_KEYS ) int [] rightJoinKeys ,
207+ @ JsonProperty (FIELD_NAME_RIGHT_UPSERT_KEY ) @ Nullable int [] rightUpsertKeys ,
189208 @ JsonProperty (FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC )
190209 DeltaJoinSpec lookupLeftTableJoinSpec ,
191210 @ JsonProperty (FIELD_NAME_INPUT_PROPERTIES ) List <InputProperty > inputProperties ,
@@ -196,8 +215,10 @@ public StreamExecDeltaJoin(
196215
197216 this .flinkJoinType = flinkJoinType ;
198217 this .leftJoinKeys = leftJoinKeys ;
218+ this .leftUpsertKeys = leftUpsertKeys ;
199219 this .lookupRightTableJoinSpec = lookupRightTableJoinSpec ;
200220 this .rightJoinKeys = rightJoinKeys ;
221+ this .rightUpsertKeys = rightUpsertKeys ;
201222 this .lookupLeftTableJoinSpec = lookupLeftTableJoinSpec ;
202223 this .asyncLookupOptions = asyncLookupOptions ;
203224 }
@@ -236,17 +257,15 @@ protected Transformation<RowData> translateToPlanInternal(
236257 RowDataKeySelector leftJoinKeySelector =
237258 KeySelectorUtil .getRowDataSelector (
238259 classLoader , leftJoinKeys , InternalTypeInfo .of (leftStreamType ));
239- // currently, delta join only supports consuming INSERT-ONLY stream
240260 RowDataKeySelector leftUpsertKeySelector =
241- getUpsertKeySelector (new int [ 0 ] , leftStreamType , classLoader );
261+ getUpsertKeySelector (leftUpsertKeys , leftStreamType , classLoader );
242262
243263 // right side selector
244264 RowDataKeySelector rightJoinKeySelector =
245265 KeySelectorUtil .getRowDataSelector (
246266 classLoader , rightJoinKeys , InternalTypeInfo .of (rightStreamType ));
247- // currently, delta join only supports consuming INSERT-ONLY stream
248267 RowDataKeySelector rightUpsertKeySelector =
249- getUpsertKeySelector (new int [ 0 ] , rightStreamType , classLoader );
268+ getUpsertKeySelector (rightUpsertKeys , rightStreamType , classLoader );
250269
251270 StreamOperatorFactory <RowData > operatorFactory =
252271 createAsyncLookupDeltaJoin (
@@ -484,9 +503,9 @@ public RexNode visitInputRef(RexInputRef inputRef) {
484503 }
485504
486505 private RowDataKeySelector getUpsertKeySelector (
487- int [] upsertKey , RowType rowType , ClassLoader classLoader ) {
506+ @ Nullable int [] upsertKey , RowType rowType , ClassLoader classLoader ) {
488507 final int [] rightUpsertKeys ;
489- if (upsertKey .length > 0 ) {
508+ if (upsertKey != null && upsertKey .length > 0 ) {
490509 rightUpsertKeys = upsertKey ;
491510 } else {
492511 rightUpsertKeys = IntStream .range (0 , rowType .getFields ().size ()).toArray ();
0 commit comments