@Test public void testLeaveGroupTimeout() throws Exception { Map<TopicPartition, Errors> response = new HashMap<>(); response.put(tp0, Errors.NONE); OffsetCommitResponse commitResponse = offsetCommitResponse(response); consumerCloseTest(5000, singletonList(commitResponse), 5000, false); }
@Test(expected = AuthenticationException.class) public void testPollAuthenticationFailure() { final KafkaConsumer<String, String> consumer = consumerWithPendingAuthentication(); consumer.subscribe(singleton(topic)); consumer.poll(Duration.ZERO); }
@Test public void verifyPollTimesOutDuringMetadataUpdate() { final Time time = new MockTime(); Metadata metadata = createMetadata(); final MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); Node node = metadata.fetch().nodes().get(0); final PartitionAssignor assignor = new RoundRobinAssignor(); final KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); prepareRebalance(client, node, assignor, singletonList(tp0), null); consumer.poll(Duration.ZERO); // The underlying client should NOT get a fetch request final Queue<ClientRequest> requests = client.requests(); Assert.assertEquals(0, requests.size()); }
private KafkaConsumer<String, String> consumerWithPendingAuthentication() { Time time = new MockTime(); Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); initMetadata(client, singletonMap(topic, 1)); Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RangeAssignor(); client.createPendingAuthenticationError(node, 0); return newConsumer(time, client, metadata, assignor, false); }
@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 shouldAttemptToRejoinGroupAfterSyncGroupFailed() 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); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, false); consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); client.prepareResponseFrom(new FindCoordinatorResponse(Errors.NONE, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); client.prepareResponseFrom(joinGroupFollowerResponse(assignor, 1, "memberId", "leaderId", Errors.NONE), coordinator); client.prepareResponseFrom(syncGroupResponse(singletonList(tp0), Errors.NONE), coordinator); client.prepareResponseFrom(fetchResponse(tp0, 0, 1), node); client.prepareResponseFrom(fetchResponse(tp0, 1, 0), node); client.prepareResponseFrom(syncGroupResponse(singletonList(tp0), Errors.NONE), coordinator, true); client.prepareResponseFrom(joinGroupFollowerResponse(assignor, 1, "memberId", "leaderId", Errors.NONE), coordinator); client.prepareResponseFrom(syncGroupResponse(singletonList(tp0), Errors.NONE), coordinator); }, fetchResponse(tp0, 1, 1), node); time.sleep(heartbeatIntervalMs); Thread.sleep(heartbeatIntervalMs);
Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 2)); KafkaConsumer<String, String> consumer = newConsumerNoAutoCommit(time, client, metadata); consumer.assign(Arrays.asList(tp0, tp1)); consumer.seekToEnd(singleton(tp0)); timestamps.get(tp1).timestamp == ListOffsetRequest.EARLIEST_TIMESTAMP; }, listOffsetsResponse(Collections.singletonMap(tp0, 50L), Collections.singletonMap(tp1, Errors.NOT_LEADER_FOR_PARTITION))); client.prepareResponse( }, fetchResponse(tp0, 50L, 5));
private KafkaConsumer<String, String> newConsumer(Time time, KafkaClient client, Metadata metadata, PartitionAssignor assignor, boolean autoCommitEnabled) { return newConsumer(time, client, metadata, assignor, OffsetResetStrategy.EARLIEST, autoCommitEnabled, groupId); }
private Node prepareRebalance(MockClient client, Node node, PartitionAssignor assignor, List<TopicPartition> partitions, Node coordinator) { if (coordinator == null) { // lookup coordinator client.prepareResponseFrom(new FindCoordinatorResponse(Errors.NONE, node), node); coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); } // join group client.prepareResponseFrom(joinGroupFollowerResponse(assignor, 1, "memberId", "leaderId", Errors.NONE), coordinator); // sync group client.prepareResponseFrom(syncGroupResponse(partitions, Errors.NONE), coordinator); return coordinator; }
@Test public void testCloseNoWait() throws Exception { consumerCloseTest(0, Collections.<AbstractResponse>emptyList(), 0, false); }
private ListOffsetResponse listOffsetsResponse(Map<TopicPartition, Long> offsets) { return listOffsetsResponse(offsets, Collections.emptyMap()); }
private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset, int count) { FetchInfo fetchInfo = new FetchInfo(fetchOffset, count); return fetchResponse(Collections.singletonMap(partition, fetchInfo)); }
@Test(expected = NoOffsetForPartitionException.class) public void testMissingOffsetNoResetPolicy() { 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, OffsetResetStrategy.NONE, true, groupId); consumer.assign(singletonList(tp0)); client.prepareResponseFrom(new FindCoordinatorResponse(Errors.NONE, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); // lookup committed offset and find nothing client.prepareResponseFrom(offsetResponse(Collections.singletonMap(tp0, -1L), Errors.NONE), coordinator); consumer.poll(Duration.ZERO); }
private KafkaConsumer<byte[], byte[]> newConsumer(String groupId) { return newConsumer(groupId, Optional.empty()); }
private Node prepareRebalance(MockClient client, Node node, final Set<String> subscribedTopics, PartitionAssignor assignor, List<TopicPartition> partitions, Node coordinator) { if (coordinator == null) { // lookup coordinator client.prepareResponseFrom(new FindCoordinatorResponse(Errors.NONE, node), node); coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); } // join group client.prepareResponseFrom(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { JoinGroupRequest joinGroupRequest = (JoinGroupRequest) body; PartitionAssignor.Subscription subscription = ConsumerProtocol.deserializeSubscription(joinGroupRequest.groupProtocols().get(0).metadata()); return subscribedTopics.equals(new HashSet<>(subscription.topics())); } }, joinGroupFollowerResponse(assignor, 1, "memberId", "leaderId", Errors.NONE), coordinator); // sync group client.prepareResponseFrom(syncGroupResponse(partitions, Errors.NONE), coordinator); return coordinator; }
@Test public void testCloseInterrupt() throws Exception { consumerCloseTest(Long.MAX_VALUE, Collections.emptyList(), 0, true); }
@SuppressWarnings("deprecation") @Test public void verifyDeprecatedPollDoesNotTimeOutDuringMetadataUpdate() { final Time time = new MockTime(); Metadata metadata = createMetadata(); final MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); Node node = metadata.fetch().nodes().get(0); final PartitionAssignor assignor = new RoundRobinAssignor(); final KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); prepareRebalance(client, node, assignor, singletonList(tp0), null); consumer.poll(0L); // The underlying client SHOULD get a fetch request final Queue<ClientRequest> requests = client.requests(); Assert.assertEquals(1, requests.size()); final Class<? extends AbstractRequest.Builder> aClass = requests.peek().requestBuilder().getClass(); Assert.assertEquals(FetchRequest.Builder.class, aClass); }
@Test public void testResetUsingAutoResetPolicy() { 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, OffsetResetStrategy.LATEST, true, groupId); consumer.assign(singletonList(tp0)); client.prepareResponseFrom(new FindCoordinatorResponse(Errors.NONE, node), node); Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); client.prepareResponseFrom(offsetResponse(Collections.singletonMap(tp0, -1L), Errors.NONE), coordinator); client.prepareResponse(listOffsetsResponse(Collections.singletonMap(tp0, 50L))); consumer.poll(Duration.ZERO); assertEquals(50L, consumer.position(tp0)); }
private KafkaConsumer<byte[], byte[]> newConsumer(String groupId, Optional<Boolean> enableAutoCommit) { Properties props = new Properties(); props.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "my.consumer"); props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); props.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); if (groupId != null) props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); if (enableAutoCommit.isPresent()) props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, enableAutoCommit.get().toString()); return newConsumer(props); }
@Test public void testGracefulClose() throws Exception { Map<TopicPartition, Errors> response = new HashMap<>(); response.put(tp0, Errors.NONE); OffsetCommitResponse commitResponse = offsetCommitResponse(response); LeaveGroupResponse leaveGroupResponse = new LeaveGroupResponse(Errors.NONE); consumerCloseTest(5000, Arrays.asList(commitResponse, leaveGroupResponse), 0, false); }