21
21
import org .apache .kafka .common .serialization .StringSerializer ;
22
22
import org .apache .kafka .common .utils .Bytes ;
23
23
import org .apache .kafka .common .utils .MockTime ;
24
+ import org .apache .kafka .streams .KeyValue ;
24
25
import org .apache .kafka .streams .StreamsBuilder ;
25
26
import org .apache .kafka .streams .StreamsConfig ;
26
27
import org .apache .kafka .streams .TestInputTopic ;
60
61
import java .util .stream .Collectors ;
61
62
import java .util .stream .Stream ;
62
63
64
+ import static java .util .Arrays .asList ;
65
+ import static java .util .Collections .emptyList ;
63
66
import static java .util .Collections .emptyMap ;
64
67
import static org .apache .kafka .common .utils .Utils .mkEntry ;
65
68
import static org .apache .kafka .common .utils .Utils .mkMap ;
66
69
import static org .apache .kafka .common .utils .Utils .mkProperties ;
70
+ import static org .hamcrest .CoreMatchers .hasItem ;
67
71
import static org .hamcrest .CoreMatchers .is ;
68
72
import static org .hamcrest .MatcherAssert .assertThat ;
69
73
@@ -183,13 +187,13 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin,
183
187
right .pipeInput ("rhs3" , "rhsValue3" , baseTimestamp + 2 ); // this unreferenced FK won't show up in any results
184
188
185
189
assertThat (
186
- outputTopic .readKeyValuesToMap (),
187
- is (emptyMap ())
190
+ outputTopic .readKeyValuesToList (),
191
+ is (emptyList ())
188
192
);
189
193
if (rejoin ) {
190
194
assertThat (
191
- rejoinOutputTopic .readKeyValuesToMap (),
192
- is (emptyMap ())
195
+ rejoinOutputTopic .readKeyValuesToList (),
196
+ is (emptyList ())
193
197
);
194
198
}
195
199
if (materialized ) {
@@ -203,27 +207,27 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin,
203
207
left .pipeInput ("lhs2" , "lhsValue2|rhs2" , baseTimestamp + 4 );
204
208
205
209
{
206
- final Map < String , String > expected = mkMap (
207
- mkEntry ("lhs1" , "(lhsValue1|rhs1,rhsValue1)" ),
208
- mkEntry ("lhs2" , "(lhsValue2|rhs2,rhsValue2)" )
210
+ final List < KeyValue < String , String >> expected = Arrays . asList (
211
+ KeyValue . pair ("lhs1" , "(lhsValue1|rhs1,rhsValue1)" ),
212
+ KeyValue . pair ("lhs2" , "(lhsValue2|rhs2,rhsValue2)" )
209
213
);
210
214
assertThat (
211
- outputTopic .readKeyValuesToMap (),
215
+ outputTopic .readKeyValuesToList (),
212
216
is (expected )
213
217
);
214
218
if (rejoin ) {
215
219
assertThat (
216
- rejoinOutputTopic .readKeyValuesToMap (),
217
- is (mkMap (
218
- mkEntry ("lhs1" , "rejoin((lhsValue1|rhs1,rhsValue1),lhsValue1|rhs1)" ),
219
- mkEntry ("lhs2" , "rejoin((lhsValue2|rhs2,rhsValue2),lhsValue2|rhs2)" )
220
+ rejoinOutputTopic .readKeyValuesToList (),
221
+ is (asList (
222
+ KeyValue . pair ("lhs1" , "rejoin((lhsValue1|rhs1,rhsValue1),lhsValue1|rhs1)" ),
223
+ KeyValue . pair ("lhs2" , "rejoin((lhsValue2|rhs2,rhsValue2),lhsValue2|rhs2)" )
220
224
))
221
225
);
222
226
}
223
227
if (materialized ) {
224
228
assertThat (
225
229
asMap (store ),
226
- is (expected )
230
+ is (expected . stream (). collect ( Collectors . toMap ( kv -> kv . key , kv -> kv . value )) )
227
231
);
228
232
}
229
233
}
@@ -232,16 +236,16 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin,
232
236
left .pipeInput ("lhs3" , "lhsValue3|rhs1" , baseTimestamp + 5 );
233
237
{
234
238
assertThat (
235
- outputTopic .readKeyValuesToMap (),
236
- is (mkMap (
237
- mkEntry ("lhs3" , "(lhsValue3|rhs1,rhsValue1)" )
239
+ outputTopic .readKeyValuesToList (),
240
+ is (List . of (
241
+ new KeyValue <> ("lhs3" , "(lhsValue3|rhs1,rhsValue1)" )
238
242
))
239
243
);
240
244
if (rejoin ) {
241
245
assertThat (
242
- rejoinOutputTopic .readKeyValuesToMap (),
243
- is (mkMap (
244
- mkEntry ("lhs3" , "rejoin((lhsValue3|rhs1,rhsValue1),lhsValue3|rhs1)" )
246
+ rejoinOutputTopic .readKeyValuesToList (),
247
+ is (List . of (
248
+ new KeyValue <> ("lhs3" , "rejoin((lhsValue3|rhs1,rhsValue1),lhsValue3|rhs1)" )
245
249
))
246
250
);
247
251
}
@@ -256,21 +260,21 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin,
256
260
);
257
261
}
258
262
}
263
+
259
264
// Now delete one LHS entity such that one delete is propagated down to the output.
260
265
261
266
left .pipeInput ("lhs1" , null , baseTimestamp + 6 );
262
267
assertThat (
263
- outputTopic .readKeyValuesToMap (),
264
- is (mkMap (
265
- mkEntry ("lhs1" , null )
268
+ outputTopic .readKeyValuesToList (),
269
+ is (List . of (
270
+ new KeyValue <> ("lhs1" , null )
266
271
))
267
272
);
268
273
if (rejoin ) {
269
274
assertThat (
270
- rejoinOutputTopic .readKeyValuesToMap (),
271
- is (mkMap (
272
- mkEntry ("lhs1" , null )
273
- ))
275
+ rejoinOutputTopic .readKeyValuesToList (),
276
+ hasItem (
277
+ KeyValue .pair ("lhs1" , null ))
274
278
);
275
279
}
276
280
if (materialized ) {
@@ -285,6 +289,79 @@ public void doJoinFromLeftThenDeleteLeftEntity(final boolean leftJoin,
285
289
}
286
290
}
287
291
292
+ @ ParameterizedTest
293
+ @ MethodSource ("testCases" )
294
+ public void doJoinFromLeftThenUpdateFkThenRevertBack (final boolean leftJoin ,
295
+ final String optimization ,
296
+ final boolean materialized ,
297
+ final boolean rejoin ,
298
+ final boolean leftVersioned ,
299
+ final boolean rightVersioned ) {
300
+ final Properties streamsConfig = getStreamsProperties (optimization );
301
+ final Topology topology = getTopology (streamsConfig , materialized ? "store" : null , leftJoin , rejoin , leftVersioned , rightVersioned );
302
+ try (final TopologyTestDriver driver = new TopologyTestDriver (topology , streamsConfig )) {
303
+ final TestInputTopic <String , String > right = driver .createInputTopic (RIGHT_TABLE , new StringSerializer (), new StringSerializer ());
304
+ final TestInputTopic <String , String > left = driver .createInputTopic (LEFT_TABLE , new StringSerializer (), new StringSerializer ());
305
+ final TestOutputTopic <String , String > outputTopic = driver .createOutputTopic (OUTPUT , new StringDeserializer (), new StringDeserializer ());
306
+ final TestOutputTopic <String , String > rejoinOutputTopic = rejoin ? driver .createOutputTopic (REJOIN_OUTPUT , new StringDeserializer (), new StringDeserializer ()) : null ;
307
+ final KeyValueStore <String , ValueAndTimestamp <String >> store = driver .getTimestampedKeyValueStore ("store" );
308
+
309
+ // Pre-populate the RHS records. This test is all about what happens when we add/remove LHS records
310
+ right .pipeInput ("rhs1" , "rhsValue1" , baseTimestamp );
311
+ right .pipeInput ("rhs2" , "rhsValue2" , baseTimestamp + 1 );
312
+
313
+ assertThat (
314
+ outputTopic .readKeyValuesToList (),
315
+ is (emptyList ())
316
+ );
317
+ if (rejoin ) {
318
+ assertThat (
319
+ rejoinOutputTopic .readKeyValuesToList (),
320
+ is (emptyList ())
321
+ );
322
+ }
323
+ if (materialized ) {
324
+ assertThat (
325
+ asMap (store ),
326
+ is (emptyMap ())
327
+ );
328
+ }
329
+
330
+ left .pipeInput ("lhs1" , "lhsValue1|rhs1" , baseTimestamp + 3 );
331
+
332
+ {
333
+ final List <KeyValue <String , String >> expected = asList (
334
+ KeyValue .pair ("lhs1" , "(lhsValue1|rhs1,rhsValue1)" )
335
+ );
336
+ assertThat (
337
+ outputTopic .readKeyValuesToList (),
338
+ is (expected )
339
+ );
340
+ }
341
+
342
+ // Add another reference to an existing FK
343
+ left .pipeInput ("lhs1" , "lhsValue1|rhs2" , baseTimestamp + 5 );
344
+ {
345
+ assertThat (
346
+ outputTopic .readKeyValuesToList (),
347
+ is (List .of (
348
+ new KeyValue <>("lhs1" , "(lhsValue1|rhs2,rhsValue2)" )
349
+ ))
350
+ );
351
+ }
352
+
353
+ // Now revert back the foreign key to earlier reference
354
+
355
+ left .pipeInput ("lhs1" , "lhsValue1|rhs1" , baseTimestamp + 6 );
356
+ assertThat (
357
+ outputTopic .readKeyValuesToList (),
358
+ is (List .of (
359
+ new KeyValue <>("lhs1" , "(lhsValue1|rhs1,rhsValue1)" )
360
+ ))
361
+ );
362
+ }
363
+ }
364
+
288
365
@ ParameterizedTest
289
366
@ MethodSource ("testCases" )
290
367
public void doJoinFromRightThenDeleteRightEntity (final boolean leftJoin ,
@@ -795,6 +872,12 @@ protected static Map<String, String> asMap(final KeyValueStore<String, ValueAndT
795
872
return result ;
796
873
}
797
874
875
+ protected static List <KeyValue <String , String >> makeList (final KeyValueStore <String , ValueAndTimestamp <String >> store ) {
876
+ final List <KeyValue <String , String >> result = new LinkedList <>();
877
+ store .all ().forEachRemaining (ele -> result .add (new KeyValue <>(ele .key , ele .value .value ())));
878
+ return result ;
879
+ }
880
+
798
881
protected static Topology getTopology (final Properties streamsConfig ,
799
882
final String queryableStoreName ,
800
883
final boolean leftJoin ,
0 commit comments