8080import org .slf4j .Logger ;
8181import org .slf4j .LoggerFactory ;
8282
83+ import javax .annotation .Nullable ;
84+
8385import java .util .Arrays ;
8486import java .util .List ;
8587import java .util .Map ;
@@ -117,6 +119,9 @@ public class StreamExecDeltaJoin extends ExecNodeBase<RowData>
117119 "lookupRightTableJoinSpec" ;
118120 private static final String FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC = "lookupLeftTableJoinSpec" ;
119121
122+ private static final String FIELD_NAME_LEFT_UPSERT_KEY = "leftUpsertKey" ;
123+ private static final String FIELD_NAME_RIGHT_UPSERT_KEY = "rightUpsertKey" ;
124+
120125 private static final String FIELD_NAME_JOIN_TYPE = "joinType" ;
121126
122127 public static final String FIELD_NAME_ASYNC_OPTIONS = "asyncOptions" ;
@@ -135,6 +140,11 @@ public class StreamExecDeltaJoin extends ExecNodeBase<RowData>
135140 @ JsonProperty (FIELD_NAME_LEFT_JOIN_KEYS )
136141 private final int [] leftJoinKeys ;
137142
143+ @ JsonProperty (FIELD_NAME_LEFT_UPSERT_KEY )
144+ @ JsonInclude (JsonInclude .Include .NON_NULL )
145+ @ Nullable
146+ private final int [] leftUpsertKeys ;
147+
138148 // left (streaming) side join right (lookup) side
139149 @ JsonProperty (FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC )
140150 private final DeltaJoinSpec lookupRightTableJoinSpec ;
@@ -144,6 +154,11 @@ public class StreamExecDeltaJoin extends ExecNodeBase<RowData>
144154 @ JsonProperty (FIELD_NAME_RIGHT_JOIN_KEYS )
145155 private final int [] rightJoinKeys ;
146156
157+ @ JsonProperty (FIELD_NAME_RIGHT_UPSERT_KEY )
158+ @ JsonInclude (JsonInclude .Include .NON_NULL )
159+ @ Nullable
160+ private final int [] rightUpsertKeys ;
161+
147162 // right (streaming) side join left (lookup) side
148163 @ JsonProperty (FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC )
149164 private final DeltaJoinSpec lookupLeftTableJoinSpec ;
@@ -153,9 +168,11 @@ public StreamExecDeltaJoin(
153168 FlinkJoinType flinkJoinType ,
154169 // delta join args related with the left side
155170 int [] leftJoinKeys ,
171+ @ Nullable int [] leftUpsertKeys ,
156172 DeltaJoinSpec lookupRightTableJoinSpec ,
157173 // delta join args related with the right side
158174 int [] rightJoinKeys ,
175+ @ Nullable int [] rightUpsertKeys ,
159176 DeltaJoinSpec lookupLeftTableJoinSpec ,
160177 InputProperty leftInputProperty ,
161178 InputProperty rightInputProperty ,
@@ -168,8 +185,10 @@ public StreamExecDeltaJoin(
168185 ExecNodeContext .newPersistedConfig (StreamExecDeltaJoin .class , tableConfig ),
169186 flinkJoinType ,
170187 leftJoinKeys ,
188+ leftUpsertKeys ,
171189 lookupRightTableJoinSpec ,
172190 rightJoinKeys ,
191+ rightUpsertKeys ,
173192 lookupLeftTableJoinSpec ,
174193 Lists .newArrayList (leftInputProperty , rightInputProperty ),
175194 outputType ,
@@ -184,9 +203,11 @@ public StreamExecDeltaJoin(
184203 @ JsonProperty (FIELD_NAME_CONFIGURATION ) ReadableConfig persistedConfig ,
185204 @ JsonProperty (FIELD_NAME_JOIN_TYPE ) FlinkJoinType flinkJoinType ,
186205 @ JsonProperty (FIELD_NAME_LEFT_JOIN_KEYS ) int [] leftJoinKeys ,
206+ @ JsonProperty (FIELD_NAME_LEFT_UPSERT_KEY ) @ Nullable int [] leftUpsertKeys ,
187207 @ JsonProperty (FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC )
188208 DeltaJoinSpec lookupRightTableJoinSpec ,
189209 @ JsonProperty (FIELD_NAME_RIGHT_JOIN_KEYS ) int [] rightJoinKeys ,
210+ @ JsonProperty (FIELD_NAME_RIGHT_UPSERT_KEY ) @ Nullable int [] rightUpsertKeys ,
190211 @ JsonProperty (FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC )
191212 DeltaJoinSpec lookupLeftTableJoinSpec ,
192213 @ JsonProperty (FIELD_NAME_INPUT_PROPERTIES ) List <InputProperty > inputProperties ,
@@ -197,8 +218,10 @@ public StreamExecDeltaJoin(
197218
198219 this .flinkJoinType = flinkJoinType ;
199220 this .leftJoinKeys = leftJoinKeys ;
221+ this .leftUpsertKeys = leftUpsertKeys ;
200222 this .lookupRightTableJoinSpec = lookupRightTableJoinSpec ;
201223 this .rightJoinKeys = rightJoinKeys ;
224+ this .rightUpsertKeys = rightUpsertKeys ;
202225 this .lookupLeftTableJoinSpec = lookupLeftTableJoinSpec ;
203226 this .asyncLookupOptions = asyncLookupOptions ;
204227 }
@@ -237,17 +260,15 @@ protected Transformation<RowData> translateToPlanInternal(
237260 RowDataKeySelector leftJoinKeySelector =
238261 KeySelectorUtil .getRowDataSelector (
239262 classLoader , leftJoinKeys , InternalTypeInfo .of (leftStreamType ));
240- // currently, delta join only supports consuming INSERT-ONLY stream
241263 RowDataKeySelector leftUpsertKeySelector =
242- getUpsertKeySelector (new int [ 0 ] , leftStreamType , classLoader );
264+ getUpsertKeySelector (leftUpsertKeys , leftStreamType , classLoader );
243265
244266 // right side selector
245267 RowDataKeySelector rightJoinKeySelector =
246268 KeySelectorUtil .getRowDataSelector (
247269 classLoader , rightJoinKeys , InternalTypeInfo .of (rightStreamType ));
248- // currently, delta join only supports consuming INSERT-ONLY stream
249270 RowDataKeySelector rightUpsertKeySelector =
250- getUpsertKeySelector (new int [ 0 ] , rightStreamType , classLoader );
271+ getUpsertKeySelector (rightUpsertKeys , rightStreamType , classLoader );
251272
252273 StreamOperatorFactory <RowData > operatorFactory =
253274 createAsyncLookupDeltaJoin (
@@ -485,9 +506,9 @@ public RexNode visitInputRef(RexInputRef inputRef) {
485506 }
486507
487508 private RowDataKeySelector getUpsertKeySelector (
488- int [] upsertKey , RowType rowType , ClassLoader classLoader ) {
509+ @ Nullable int [] upsertKey , RowType rowType , ClassLoader classLoader ) {
489510 final int [] rightUpsertKeys ;
490- if (upsertKey .length > 0 ) {
511+ if (upsertKey != null && upsertKey .length > 0 ) {
491512 rightUpsertKeys = upsertKey ;
492513 } else {
493514 rightUpsertKeys = IntStream .range (0 , rowType .getFields ().size ()).toArray ();
0 commit comments