private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset, int count) { FetchInfo fetchInfo = new FetchInfo(fetchOffset, count); return fetchResponse(Collections.singletonMap(partition, fetchInfo)); }
@Test public void verifyHeartbeatSentWhenFetchedDataReady() throws Exception { Time time = new MockTime(); Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RoundRobinAssignor(); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); Node coordinator = prepareRebalance(client, node, assignor, singletonList(tp0), null); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); consumer.poll(Duration.ZERO); // respond to the outstanding fetch so that we have data available on the next poll client.respondFrom(fetchResponse(tp0, 0, 5), node); client.poll(0, time.milliseconds()); client.prepareResponseFrom(fetchResponse(tp0, 5, 0), node); AtomicBoolean heartbeatReceived = prepareHeartbeatResponse(client, coordinator); time.sleep(heartbeatIntervalMs); Thread.sleep(heartbeatIntervalMs); consumer.poll(Duration.ZERO); assertTrue(heartbeatReceived.get()); consumer.close(Duration.ofMillis(0)); }
}, fetchResponse(tp0, 50L, 5));
@Test public void fetchResponseWithUnexpectedPartitionIsIgnored() { Time time = new MockTime(); Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RangeAssignor(); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); consumer.subscribe(singletonList(topic), getConsumerRebalanceListener(consumer)); prepareRebalance(client, node, assignor, singletonList(tp0), null); Map<TopicPartition, FetchInfo> fetches1 = new HashMap<>(); fetches1.put(tp0, new FetchInfo(0, 1)); fetches1.put(t2p0, new FetchInfo(0, 10)); // not assigned and not fetched client.prepareResponseFrom(fetchResponse(fetches1), node); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); ConsumerRecords<String, String> records = consumer.poll(Duration.ZERO); assertEquals(0, records.count()); consumer.close(Duration.ofMillis(0)); }
@Test public void verifyNoCoordinatorLookupForManualAssignmentWithSeek() { Time time = new MockTime(); Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); PartitionAssignor assignor = new RoundRobinAssignor(); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); consumer.assign(singleton(tp0)); consumer.seekToBeginning(singleton(tp0)); // there shouldn't be any need to lookup the coordinator or fetch committed offsets. // we just lookup the starting position and send the record fetch. client.prepareResponse(listOffsetsResponse(Collections.singletonMap(tp0, 50L))); client.prepareResponse(fetchResponse(tp0, 50L, 5)); ConsumerRecords<String, String> records = consumer.poll(Duration.ofMillis(1)); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); consumer.close(Duration.ofMillis(0)); }
@Test public void testAutoCommitSentBeforePositionUpdate() { Time time = new MockTime(); Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RoundRobinAssignor(); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); Node coordinator = prepareRebalance(client, node, assignor, singletonList(tp0), null); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); consumer.poll(Duration.ZERO); // respond to the outstanding fetch so that we have data available on the next poll client.respondFrom(fetchResponse(tp0, 0, 5), node); client.poll(0, time.milliseconds()); time.sleep(autoCommitIntervalMs); client.prepareResponseFrom(fetchResponse(tp0, 5, 0), node); // no data has been returned to the user yet, so the committed offset should be 0 AtomicBoolean commitReceived = prepareOffsetCommitResponse(client, coordinator, tp0, 0); consumer.poll(Duration.ZERO); assertTrue(commitReceived.get()); consumer.close(Duration.ofMillis(0)); }
client.prepareResponseFrom(fetchResponse(tp0, 0, 1), node); client.prepareResponseFrom(fetchResponse(tp0, 1, 0), node); consumer.poll(Duration.ZERO);
@Test public void verifyHeartbeatSent() throws Exception { Time time = new MockTime(); Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RoundRobinAssignor(); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); Node coordinator = prepareRebalance(client, node, assignor, singletonList(tp0), null); // initial fetch client.prepareResponseFrom(fetchResponse(tp0, 0, 0), node); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); assertEquals(singleton(tp0), consumer.assignment()); AtomicBoolean heartbeatReceived = prepareHeartbeatResponse(client, coordinator); // heartbeat interval is 2 seconds time.sleep(heartbeatIntervalMs); Thread.sleep(heartbeatIntervalMs); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); assertTrue(heartbeatReceived.get()); consumer.close(Duration.ofMillis(0)); }
client.respondFrom(fetchResponse(tp0, 0, 5), node); client.poll(0, time.milliseconds());
client.prepareResponse(fetchResponse(tp0, 10L, 1));
fetches1.put(tp0, new FetchInfo(0, 1)); fetches1.put(t2p0, new FetchInfo(0, 10)); client.respondFrom(fetchResponse(fetches1), node); client.poll(0, time.milliseconds()); client.respondFrom(fetchResponse(fetches1), node); client.poll(0, time.milliseconds()); fetches2.put(tp0, new FetchInfo(1, 1)); fetches2.put(t3p0, new FetchInfo(0, 100)); client.prepareResponse(fetchResponse(fetches2));
client.prepareResponse(fetchResponse(tp0, 10L, 1));
client.prepareResponseFrom(syncGroupResponse(singletonList(tp0), Errors.NONE), coordinator); client.prepareResponseFrom(fetchResponse(tp0, 0, 1), node); client.prepareResponseFrom(fetchResponse(tp0, 1, 0), node); return body instanceof FetchRequest && ((FetchRequest) body).fetchData().containsKey(tp0); }, fetchResponse(tp0, 1, 1), node); time.sleep(heartbeatIntervalMs); Thread.sleep(heartbeatIntervalMs);