27
27
import org .apache .kafka .common .serialization .Deserializer ;
28
28
import org .apache .kafka .common .serialization .IntegerDeserializer ;
29
29
import org .apache .kafka .common .serialization .IntegerSerializer ;
30
- import org .apache .kafka .common .serialization .Serdes ;
30
+ import org .apache .kafka .common .serialization .LongSerializer ;
31
31
import org .apache .kafka .common .serialization .Serializer ;
32
32
import org .apache .kafka .common .utils .Bytes ;
33
33
import org .apache .kafka .common .utils .LogContext ;
@@ -75,8 +75,7 @@ public class RecordQueueTest {
75
75
private final StreamsMetricsImpl streamsMetrics =
76
76
new StreamsMetricsImpl (metrics , "mock" , StreamsConfig .METRICS_LATEST , new MockTime ());
77
77
78
- @ SuppressWarnings ("rawtypes" )
79
- final InternalMockProcessorContext context = new InternalMockProcessorContext <>(
78
+ final InternalMockProcessorContext <Integer , Integer > context = new InternalMockProcessorContext <>(
80
79
StateSerdes .withBuiltinTypes ("anyName" , Bytes .class , Bytes .class ),
81
80
new MockRecordCollector (),
82
81
metrics
@@ -89,19 +88,28 @@ public class RecordQueueTest {
89
88
timestampExtractor ,
90
89
new LogAndFailExceptionHandler (),
91
90
context ,
92
- new LogContext ());
91
+ new LogContext ()
92
+ );
93
93
private final RecordQueue queueThatSkipsDeserializeErrors = new RecordQueue (
94
94
new TopicPartition ("topic" , 1 ),
95
95
mockSourceNodeWithMetrics ,
96
96
timestampExtractor ,
97
97
new LogAndContinueExceptionHandler (),
98
98
context ,
99
- new LogContext ());
99
+ new LogContext ()
100
+ );
101
+ private final RecordQueue queueThatSkipsInvalidTimestamps = new RecordQueue (
102
+ new TopicPartition ("topic" , 1 ),
103
+ mockSourceNodeWithMetrics ,
104
+ new LogAndSkipOnInvalidTimestamp (),
105
+ new LogAndFailExceptionHandler (),
106
+ context ,
107
+ new LogContext ()
108
+ );
100
109
101
110
private final byte [] recordValue = intSerializer .serialize (null , 10 );
102
111
private final byte [] recordKey = intSerializer .serialize (null , 1 );
103
112
104
- @ SuppressWarnings ("unchecked" )
105
113
@ BeforeEach
106
114
public void before () {
107
115
mockSourceNodeWithMetrics .init (context );
@@ -328,7 +336,7 @@ public void shouldSetTimestampAndRespectMaxTimestampPolicy() {
328
336
329
337
@ Test
330
338
public void shouldThrowStreamsExceptionWhenKeyDeserializationFails () {
331
- final byte [] key = Serdes . Long (). serializer ().serialize ("foo" , 1L );
339
+ final byte [] key = new LongSerializer ().serialize ("foo" , 1L );
332
340
final List <ConsumerRecord <byte [], byte []>> records = Collections .singletonList (
333
341
new ConsumerRecord <>("topic" , 1 , 1 , 0L , TimestampType .CREATE_TIME , 0 , 0 , key , recordValue ,
334
342
new RecordHeaders (), Optional .empty ()));
@@ -342,7 +350,7 @@ public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() {
342
350
343
351
@ Test
344
352
public void shouldThrowStreamsExceptionWhenValueDeserializationFails () {
345
- final byte [] value = Serdes . Long (). serializer ().serialize ("foo" , 1L );
353
+ final byte [] value = new LongSerializer ().serialize ("foo" , 1L );
346
354
final List <ConsumerRecord <byte [], byte []>> records = Collections .singletonList (
347
355
new ConsumerRecord <>("topic" , 1 , 1 , 0L , TimestampType .CREATE_TIME , 0 , 0 , recordKey , value ,
348
356
new RecordHeaders (), Optional .empty ()));
@@ -356,7 +364,7 @@ public void shouldThrowStreamsExceptionWhenValueDeserializationFails() {
356
364
357
365
@ Test
358
366
public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler () {
359
- final byte [] key = Serdes . Long (). serializer ().serialize ("foo" , 1L );
367
+ final byte [] key = new LongSerializer ().serialize ("foo" , 1L );
360
368
final ConsumerRecord <byte [], byte []> record = new ConsumerRecord <>("topic" , 1 , 1 , 0L ,
361
369
TimestampType .CREATE_TIME , 0 , 0 , key , recordValue ,
362
370
new RecordHeaders (), Optional .empty ());
@@ -369,7 +377,7 @@ public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHan
369
377
370
378
@ Test
371
379
public void shouldNotThrowStreamsExceptionWhenValueDeserializationFailsWithSkipHandler () {
372
- final byte [] value = Serdes . Long (). serializer ().serialize ("foo" , 1L );
380
+ final byte [] value = new LongSerializer ().serialize ("foo" , 1L );
373
381
final ConsumerRecord <byte [], byte []> record = new ConsumerRecord <>("topic" , 1 , 1 , 0L ,
374
382
TimestampType .CREATE_TIME , 0 , 0 , recordKey , value ,
375
383
new RecordHeaders (), Optional .empty ());
@@ -392,7 +400,7 @@ public void shouldThrowOnNegativeTimestamp() {
392
400
mockSourceNodeWithMetrics ,
393
401
new FailOnInvalidTimestamp (),
394
402
new LogAndContinueExceptionHandler (),
395
- new InternalMockProcessorContext (),
403
+ new InternalMockProcessorContext <> (),
396
404
new LogContext ());
397
405
398
406
final StreamsException exception = assertThrows (
@@ -409,20 +417,25 @@ public void shouldThrowOnNegativeTimestamp() {
409
417
410
418
@ Test
411
419
public void shouldDropOnNegativeTimestamp () {
412
- final List <ConsumerRecord <byte [], byte []>> records = Collections .singletonList (
413
- new ConsumerRecord <>("topic" , 1 , 1 , -1L , TimestampType .CREATE_TIME , 0 , 0 , recordKey , recordValue ,
414
- new RecordHeaders (), Optional .empty ()));
420
+ final ConsumerRecord <byte [], byte []> record = new ConsumerRecord <>(
421
+ "topic" ,
422
+ 1 ,
423
+ 1 ,
424
+ -1L , // negative timestamp
425
+ TimestampType .CREATE_TIME ,
426
+ 0 ,
427
+ 0 ,
428
+ recordKey ,
429
+ recordValue ,
430
+ new RecordHeaders (),
431
+ Optional .empty ()
432
+ );
433
+ final List <ConsumerRecord <byte [], byte []>> records = Collections .singletonList (record );
415
434
416
- final RecordQueue queue = new RecordQueue (
417
- new TopicPartition ("topic" , 1 ),
418
- mockSourceNodeWithMetrics ,
419
- new LogAndSkipOnInvalidTimestamp (),
420
- new LogAndContinueExceptionHandler (),
421
- new InternalMockProcessorContext (),
422
- new LogContext ());
423
- queue .addRawRecords (records );
435
+ queueThatSkipsInvalidTimestamps .addRawRecords (records );
424
436
425
- assertEquals (0 , queue .size ());
437
+ assertEquals (1 , queueThatSkipsInvalidTimestamps .size ());
438
+ assertEquals (new CorruptedRecord (record ), queueThatSkipsInvalidTimestamps .poll (0 ));
426
439
}
427
440
428
441
@ Test
0 commit comments