26
26
import java .util .Arrays ;
27
27
import java .util .Collection ;
28
28
import java .util .Collections ;
29
+ import java .util .HashMap ;
29
30
import java .util .Iterator ;
30
31
import java .util .LinkedHashMap ;
31
32
import java .util .List ;
@@ -81,6 +82,7 @@ public void testRecordsByPartition() {
81
82
82
83
ConsumerRecords <Integer , String > consumerRecords = buildTopicTestRecords (recordSize , partitionSize , emptyPartitionIndex , topics );
83
84
85
+ assertEquals (partitionSize * topics .size (), consumerRecords .nextOffsets ().size ());
84
86
for (String topic : topics ) {
85
87
for (int partition = 0 ; partition < partitionSize ; partition ++) {
86
88
TopicPartition topicPartition = new TopicPartition (topic , partition );
@@ -90,6 +92,8 @@ public void testRecordsByPartition() {
90
92
assertTrue (records .isEmpty ());
91
93
} else {
92
94
assertEquals (recordSize , records .size ());
95
+ final ConsumerRecord <Integer , String > lastRecord = records .get (recordSize - 1 );
96
+ assertEquals (new OffsetAndMetadata (lastRecord .offset () + 1 , lastRecord .leaderEpoch (), "" ), consumerRecords .nextOffsets ().get (topicPartition ));
93
97
for (int i = 0 ; i < records .size (); i ++) {
94
98
ConsumerRecord <Integer , String > record = records .get (i );
95
99
validateRecordPayload (topic , record , partition , i , recordSize );
@@ -117,6 +121,8 @@ public void testRecordsByTopic() {
117
121
118
122
ConsumerRecords <Integer , String > consumerRecords = buildTopicTestRecords (recordSize , partitionSize , emptyPartitionIndex , topics );
119
123
124
+ assertEquals (partitionSize * topics .size (), consumerRecords .nextOffsets ().size ());
125
+
120
126
for (String topic : topics ) {
121
127
Iterable <ConsumerRecord <Integer , String >> records = consumerRecords .records (topic );
122
128
int recordCount = 0 ;
@@ -156,6 +162,7 @@ public void testRecordsAreImmutable() {
156
162
ConsumerRecords <Integer , String > records = buildTopicTestRecords (recordSize , partitionSize , emptyPartitionIndex , Collections .singleton (topic ));
157
163
ConsumerRecords <Integer , String > emptyRecords = ConsumerRecords .empty ();
158
164
165
+ assertEquals (partitionSize , records .nextOffsets ().size ());
159
166
// check records(TopicPartition) / partitions by add method
160
167
// check iterator / records(String) by remove method
161
168
// check data count after all operations
@@ -178,6 +185,7 @@ private ConsumerRecords<Integer, String> buildTopicTestRecords(int recordSize,
178
185
int emptyPartitionIndex ,
179
186
Collection <String > topics ) {
180
187
Map <TopicPartition , List <ConsumerRecord <Integer , String >>> partitionToRecords = new LinkedHashMap <>();
188
+ Map <TopicPartition , OffsetAndMetadata > nextOffsets = new HashMap <>();
181
189
for (String topic : topics ) {
182
190
for (int i = 0 ; i < partitionSize ; i ++) {
183
191
List <ConsumerRecord <Integer , String >> records = new ArrayList <>(recordSize );
@@ -189,11 +197,13 @@ private ConsumerRecords<Integer, String> buildTopicTestRecords(int recordSize,
189
197
);
190
198
}
191
199
}
192
- partitionToRecords .put (new TopicPartition (topic , i ), records );
200
+ final TopicPartition tp = new TopicPartition (topic , i );
201
+ partitionToRecords .put (tp , records );
202
+ nextOffsets .put (tp , new OffsetAndMetadata (recordSize , Optional .empty (), "" ));
193
203
}
194
204
}
195
205
196
- return new ConsumerRecords <>(partitionToRecords );
206
+ return new ConsumerRecords <>(partitionToRecords , nextOffsets );
197
207
}
198
208
199
209
private void validateEmptyPartition (ConsumerRecord <Integer , String > record , int emptyPartitionIndex ) {
0 commit comments