@@ -88,31 +88,40 @@ public void testIncreasingRecordBasedCounters() throws Exception {
88
88
89
89
try (final KafkaWriter <Integer > writer =
90
90
createWriterWithConfiguration (
91
- getKafkaClientConfiguration (), DeliveryGuarantee .NONE , metricGroup )) {
91
+ getKafkaClientConfiguration (),
92
+ DeliveryGuarantee .AT_LEAST_ONCE ,
93
+ metricGroup )) {
92
94
final Counter numBytesOut = metricGroup .getIOMetricGroup ().getNumBytesOutCounter ();
93
95
final Counter numRecordsOut = metricGroup .getIOMetricGroup ().getNumRecordsOutCounter ();
94
96
final Counter numRecordsOutErrors = metricGroup .getNumRecordsOutErrorsCounter ();
95
97
final Counter numRecordsSendErrors = metricGroup .getNumRecordsSendErrorsCounter ();
96
- assertThat (numBytesOut .getCount ()).isEqualTo (0L );
97
- assertThat (numRecordsOut .getCount ()).isEqualTo (0 );
98
+
99
+ // ApiVersionsRequest etc. is sent on initialization, so establish some baseline
100
+ writer .write (0 , SINK_WRITER_CONTEXT );
101
+ writer .flush (false ); // ensure data is actually written
102
+ timeService .trigger (); // sync byte count
103
+ long baselineCount = numBytesOut .getCount ();
104
+ assertThat (numRecordsOut .getCount ()).isEqualTo (1 );
98
105
assertThat (numRecordsOutErrors .getCount ()).isEqualTo (0 );
99
106
assertThat (numRecordsSendErrors .getCount ()).isEqualTo (0 );
100
107
101
108
// elements for which the serializer returns null should be silently skipped
102
109
writer .write (null , SINK_WRITER_CONTEXT );
103
- timeService .trigger ();
104
- assertThat (numBytesOut .getCount ()).isEqualTo (0L );
105
- assertThat (numRecordsOut .getCount ()).isEqualTo (0 );
110
+ writer .flush (false );
111
+ timeService .trigger (); // sync byte count
112
+ assertThat (numBytesOut .getCount ()).isEqualTo (baselineCount );
113
+ assertThat (numRecordsOut .getCount ()).isEqualTo (1 );
106
114
assertThat (numRecordsOutErrors .getCount ()).isEqualTo (0 );
107
115
assertThat (numRecordsSendErrors .getCount ()).isEqualTo (0 );
108
116
109
117
// but elements for which a non-null producer record is returned should count
110
118
writer .write (1 , SINK_WRITER_CONTEXT );
111
- timeService .trigger ();
112
- assertThat (numRecordsOut .getCount ()).isEqualTo (1 );
119
+ writer .flush (false );
120
+ timeService .trigger (); // sync byte count
121
+ assertThat (numBytesOut .getCount ()).isGreaterThan (baselineCount );
122
+ assertThat (numRecordsOut .getCount ()).isEqualTo (2 );
113
123
assertThat (numRecordsOutErrors .getCount ()).isEqualTo (0 );
114
124
assertThat (numRecordsSendErrors .getCount ()).isEqualTo (0 );
115
- assertThat (numBytesOut .getCount ()).isGreaterThan (0L );
116
125
}
117
126
}
118
127
0 commit comments