/** * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * <p> * This is a short-hand for {@link #subscribe(Pattern, ConsumerRebalanceListener)}, which * uses a no-op listener. If you need the ability to seek to particular offsets, you should prefer * {@link #subscribe(Pattern, ConsumerRebalanceListener)}, since group rebalances will cause partition offsets * to be reset. You should also provide your own listener if you are doing your own offset * management since the listener gives you an opportunity to commit offsets before a rebalance finishes. * * @param pattern Pattern to subscribe to * @throws IllegalArgumentException If pattern is null * @throws IllegalStateException If {@code subscribe()} is called previously with topics, or assign is called * previously (without a subsequent call to {@link #unsubscribe()}), or if not * configured at-least one partition assignment strategy */ @Override public void subscribe(Pattern pattern) { subscribe(pattern, new NoOpConsumerRebalanceListener()); }
/** * Subscribe to the given list of topics to get dynamically assigned partitions. * <b>Topic subscriptions are not incremental. This list will replace the current * assignment (if there is one).</b> It is not possible to combine topic subscription with group management * with manual partition assignment through {@link #assign(Collection)}. * * If the given list of topics is empty, it is treated the same as {@link #unsubscribe()}. * * <p> * This is a short-hand for {@link #subscribe(Collection, ConsumerRebalanceListener)}, which * uses a no-op listener. If you need the ability to seek to particular offsets, you should prefer * {@link #subscribe(Collection, ConsumerRebalanceListener)}, since group rebalances will cause partition offsets * to be reset. You should also provide your own listener if you are doing your own offset * management since the listener gives you an opportunity to commit offsets before a rebalance finishes. * * @param topics The list of topics to subscribe to * @throws IllegalArgumentException If topics is null or contains null or empty elements * @throws IllegalStateException If {@code subscribe()} is called previously with pattern, or assign is called * previously (without a subsequent call to {@link #unsubscribe()}), or if not * configured at-least one partition assignment strategy */ @Override public void subscribe(Collection<String> topics) { subscribe(topics, new NoOpConsumerRebalanceListener()); }
@Test(expected = IllegalStateException.class) public void testSubscriptionWithEmptyPartitionAssignment() { Properties props = new Properties(); props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); props.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, ""); props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(props)) { consumer.subscribe(singletonList(topic)); } }
@Test(expected = AuthenticationException.class) public void testPollAuthenticationFailure() { final KafkaConsumer<String, String> consumer = consumerWithPendingAuthentication(); consumer.subscribe(singleton(topic)); consumer.poll(Duration.ZERO); }
@Test(expected = IllegalStateException.class) public void testPollWithEmptySubscription() { try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) { consumer.subscribe(Collections.<String>emptyList()); consumer.poll(Duration.ZERO); } }
@Test(expected = IllegalArgumentException.class) public void testSubscriptionOnNullTopic() { try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) { consumer.subscribe(singletonList((String) null)); } }
@Test(expected = IllegalArgumentException.class) public void testSubscriptionOnEmptyTopic() { try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) { String emptyTopic = " "; consumer.subscribe(singletonList(emptyTopic)); } }
@Test public void testConsumerWithInvalidCredentials() { Map<String, Object> props = new HashMap<>(saslClientConfigs); props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + server.port()); props.put(ConsumerConfig.GROUP_ID_CONFIG, ""); StringDeserializer deserializer = new StringDeserializer(); try (KafkaConsumer<String, String> consumer = new KafkaConsumer<>(props, deserializer, deserializer)) { consumer.subscribe(Arrays.asList(topic)); consumer.poll(Duration.ofSeconds(10)); fail("Expected an authentication error!"); } catch (SaslAuthenticationException e) { // OK } catch (Exception e) { throw new AssertionError("Expected only an authentication error, but another error occurred.", e); } }
@Test(expected = IllegalArgumentException.class) public void testSubscriptionOnNullPattern() { try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) { consumer.subscribe((Pattern) null); } }
@Test(expected = IllegalArgumentException.class) public void testSubscriptionOnNullTopicCollection() { try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) { consumer.subscribe((List<String>) null); } }
@Test public void testSubscription() { KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId); consumer.subscribe(singletonList(topic)); assertEquals(singleton(topic), consumer.subscription()); assertTrue(consumer.assignment().isEmpty()); consumer.subscribe(Collections.<String>emptyList()); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); consumer.assign(singletonList(tp0)); assertTrue(consumer.subscription().isEmpty()); assertEquals(singleton(tp0), consumer.assignment()); consumer.unsubscribe(); assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); consumer.close(); }
@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()); }
@Test public void testPollThrowsInterruptExceptionIfInterrupted() { final Time time = new MockTime(); final 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(); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, false); consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); prepareRebalance(client, node, assignor, singletonList(tp0), null); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); consumer.poll(Duration.ZERO); // interrupt the thread and call poll try { Thread.currentThread().interrupt(); expectedException.expect(InterruptException.class); consumer.poll(Duration.ZERO); } finally { // clear interrupted state again since this thread may be reused by JUnit Thread.interrupted(); } consumer.close(Duration.ofMillis(0)); }
@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 testRegexSubscription() { String unmatchedTopic = "unmatched"; Time time = new MockTime(); Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); Map<String, Integer> partitionCounts = new HashMap<>(); partitionCounts.put(topic, 1); partitionCounts.put(unmatchedTopic, 1); initMetadata(client, partitionCounts); Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RoundRobinAssignor(); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); prepareRebalance(client, node, singleton(topic), assignor, singletonList(tp0), null); consumer.subscribe(Pattern.compile(topic), getConsumerRebalanceListener(consumer)); client.prepareMetadataUpdate(TestUtils.metadataUpdateWith(1, partitionCounts)); consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); assertEquals(singleton(topic), consumer.subscription()); assertEquals(singleton(tp0), consumer.assignment()); consumer.close(Duration.ofMillis(0)); }
@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(expected = InvalidTopicException.class) public void testSubscriptionOnInvalidTopic() { Time time = new MockTime(); Metadata metadata = createMetadata(); MockClient client = new MockClient(time, metadata); initMetadata(client, Collections.singletonMap(topic, 1)); Cluster cluster = metadata.fetch(); PartitionAssignor assignor = new RoundRobinAssignor(); String invalidTopicName = "topic abc"; // Invalid topic name due to space List<MetadataResponse.TopicMetadata> topicMetadata = new ArrayList<>(); topicMetadata.add(new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION, invalidTopicName, false, Collections.emptyList())); MetadataResponse updateResponse = new MetadataResponse(cluster.nodes(), cluster.clusterResource().clusterId(), cluster.controller().id(), topicMetadata); client.prepareMetadataUpdate(updateResponse); KafkaConsumer<String, String> consumer = newConsumer(time, client, metadata, assignor, true); consumer.subscribe(singleton(invalidTopicName), getConsumerRebalanceListener(consumer)); consumer.poll(Duration.ZERO); } }
@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)); }
@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)); }
@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)); }