37
37
import java .time .Duration ;
38
38
import java .util .Properties ;
39
39
import java .util .concurrent .Future ;
40
+ import java .util .concurrent .atomic .AtomicLong ;
40
41
41
42
import static org .apache .flink .util .Preconditions .checkState ;
42
43
@@ -55,6 +56,7 @@ class FlinkKafkaInternalProducer<K, V> extends KafkaProducer<K, V> {
55
56
private volatile boolean inTransaction ;
56
57
private volatile boolean hasRecordsInTransaction ;
57
58
private volatile boolean closed ;
59
+ private final AtomicLong pendingRecords = new AtomicLong (0 );
58
60
59
61
public FlinkKafkaInternalProducer (Properties properties , @ Nullable String transactionalId ) {
60
62
super (withTransactionalId (properties , transactionalId ));
@@ -72,12 +74,17 @@ private static Properties withTransactionalId(
72
74
return props ;
73
75
}
74
76
77
+ public long getPendingRecordsCount () {
78
+ return pendingRecords .get ();
79
+ }
80
+
75
81
@ Override
76
82
public Future <RecordMetadata > send (ProducerRecord <K , V > record , Callback callback ) {
77
83
if (inTransaction ) {
78
84
hasRecordsInTransaction = true ;
79
85
}
80
- return super .send (record , callback );
86
+ pendingRecords .incrementAndGet ();
87
+ return super .send (record , new TrackingCallback (callback ));
81
88
}
82
89
83
90
@ Override
@@ -86,6 +93,11 @@ public void flush() {
86
93
if (inTransaction ) {
87
94
flushNewPartitions ();
88
95
}
96
+ final long pendingRecordsCount = pendingRecords .get ();
97
+ if (pendingRecordsCount != 0 ) {
98
+ throw new IllegalStateException (
99
+ "Pending record count must be zero at this point: " + pendingRecordsCount );
100
+ }
89
101
}
90
102
91
103
@ Override
@@ -396,8 +408,27 @@ public String toString() {
396
408
+ transactionalId
397
409
+ "', inTransaction="
398
410
+ inTransaction
411
+ + ", pendingRecords="
412
+ + pendingRecords .get ()
399
413
+ ", closed="
400
414
+ closed
401
415
+ '}' ;
402
416
}
417
+
418
+ public class TrackingCallback implements Callback {
419
+
420
+ private final Callback actualCallback ;
421
+
422
+ public TrackingCallback (final Callback actualCallback ) {
423
+ this .actualCallback = actualCallback ;
424
+ }
425
+
426
+ @ Override
427
+ public void onCompletion (final RecordMetadata recordMetadata , final Exception e ) {
428
+ pendingRecords .decrementAndGet ();
429
+ if (actualCallback != null ) {
430
+ actualCallback .onCompletion (recordMetadata , e );
431
+ }
432
+ }
433
+ }
403
434
}
0 commit comments