private KafkaConsumer<String, String> newConsumerNoAutoCommit(Time time, KafkaClient client, Metadata metadata) { return newConsumer(time, client, metadata, new RangeAssignor(), OffsetResetStrategy.EARLIEST, false, groupId); }
@Test public void testOneConsumerNoTopic() { String consumerId = "consumer"; Map<String, Integer> partitionsPerTopic = new HashMap<>(); Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, Collections.singletonMap(consumerId, new Subscription(Collections.<String>emptyList()))); assertEquals(Collections.singleton(consumerId), assignment.keySet()); assertTrue(assignment.get(consumerId).isEmpty()); }
@Override public Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic, Map<String, Subscription> subscriptions) { Map<String, List<String>> consumersPerTopic = consumersPerTopic(subscriptions); Map<String, List<TopicPartition>> assignment = new HashMap<>(); for (String memberId : subscriptions.keySet()) assignment.put(memberId, new ArrayList<>()); for (Map.Entry<String, List<String>> topicEntry : consumersPerTopic.entrySet()) { String topic = topicEntry.getKey(); List<String> consumersForTopic = topicEntry.getValue(); Integer numPartitionsForTopic = partitionsPerTopic.get(topic); if (numPartitionsForTopic == null) continue; Collections.sort(consumersForTopic); int numPartitionsPerConsumer = numPartitionsForTopic / consumersForTopic.size(); int consumersWithExtraPartition = numPartitionsForTopic % consumersForTopic.size(); List<TopicPartition> partitions = AbstractPartitionAssignor.partitions(topic, numPartitionsForTopic); for (int i = 0, n = consumersForTopic.size(); i < n; i++) { int start = numPartitionsPerConsumer * i + Math.min(i, consumersWithExtraPartition); int length = numPartitionsPerConsumer + (i + 1 > consumersWithExtraPartition ? 0 : 1); assignment.get(consumersForTopic.get(i)).addAll(partitions.subList(start, start + length)); } } return assignment; }
@Test public void testProtocolMetadataOrder() { RoundRobinAssignor roundRobin = new RoundRobinAssignor(); RangeAssignor range = new RangeAssignor(); try (Metrics metrics = new Metrics(time)) { ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.<PartitionAssignor>asList(roundRobin, range), ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, false, true); List<ProtocolMetadata> metadata = coordinator.metadata(); assertEquals(2, metadata.size()); assertEquals(roundRobin.name(), metadata.get(0).name()); assertEquals(range.name(), metadata.get(1).name()); } try (Metrics metrics = new Metrics(time)) { ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.<PartitionAssignor>asList(range, roundRobin), ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, false, true); List<ProtocolMetadata> metadata = coordinator.metadata(); assertEquals(2, metadata.size()); assertEquals(range.name(), metadata.get(0).name()); assertEquals(roundRobin.name(), metadata.get(1).name()); } }
private Map<String, List<String>> consumersPerTopic(Map<String, Subscription> consumerMetadata) { Map<String, List<String>> res = new HashMap<>(); for (Map.Entry<String, Subscription> subscriptionEntry : consumerMetadata.entrySet()) { String consumerId = subscriptionEntry.getKey(); for (String topic : subscriptionEntry.getValue().topics()) put(res, topic, consumerId); } return res; }
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 testOneConsumerNonexistentTopic() { String topic = "topic"; String consumerId = "consumer"; Map<String, Integer> partitionsPerTopic = new HashMap<>(); Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, Collections.singletonMap(consumerId, new Subscription(topics(topic)))); assertEquals(Collections.singleton(consumerId), assignment.keySet()); assertTrue(assignment.get(consumerId).isEmpty()); }
@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 testTwoConsumersOneTopicOnePartition() { String topic = "topic"; String consumer1 = "consumer1"; String consumer2 = "consumer2"; Map<String, Integer> partitionsPerTopic = new HashMap<>(); partitionsPerTopic.put(topic, 1); Map<String, Subscription> consumers = new HashMap<>(); consumers.put(consumer1, new Subscription(topics(topic))); consumers.put(consumer2, new Subscription(topics(topic))); Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers); assertAssignment(partitions(tp(topic, 0)), assignment.get(consumer1)); assertAssignment(Collections.<TopicPartition>emptyList(), assignment.get(consumer2)); }
Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RangeAssignor();
@Test public void testOnlyAssignsPartitionsFromSubscribedTopics() { String topic = "topic"; String otherTopic = "other"; String consumerId = "consumer"; Map<String, Integer> partitionsPerTopic = new HashMap<>(); partitionsPerTopic.put(topic, 3); partitionsPerTopic.put(otherTopic, 3); Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, Collections.singletonMap(consumerId, new Subscription(topics(topic)))); assertEquals(Collections.singleton(consumerId), assignment.keySet()); assertAssignment(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId)); }
Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RangeAssignor();
@Test public void testOneConsumerMultipleTopics() { String topic1 = "topic1"; String topic2 = "topic2"; String consumerId = "consumer"; Map<String, Integer> partitionsPerTopic = new HashMap<>(); partitionsPerTopic.put(topic1, 1); partitionsPerTopic.put(topic2, 2); Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, Collections.singletonMap(consumerId, new Subscription(topics(topic1, topic2)))); assertEquals(Collections.singleton(consumerId), assignment.keySet()); assertAssignment(partitions(tp(topic1, 0), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumerId)); }
Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RangeAssignor();
@Test public void testOneConsumerOneTopic() { String topic = "topic"; String consumerId = "consumer"; Map<String, Integer> partitionsPerTopic = new HashMap<>(); partitionsPerTopic.put(topic, 3); Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, Collections.singletonMap(consumerId, new Subscription(topics(topic)))); assertEquals(Collections.singleton(consumerId), assignment.keySet()); assertAssignment(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId)); }
Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RangeAssignor();
@Test public void testTwoConsumersOneTopicTwoPartitions() { String topic = "topic"; String consumer1 = "consumer1"; String consumer2 = "consumer2"; Map<String, Integer> partitionsPerTopic = new HashMap<>(); partitionsPerTopic.put(topic, 2); Map<String, Subscription> consumers = new HashMap<>(); consumers.put(consumer1, new Subscription(topics(topic))); consumers.put(consumer2, new Subscription(topics(topic))); Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers); assertAssignment(partitions(tp(topic, 0)), assignment.get(consumer1)); assertAssignment(partitions(tp(topic, 1)), assignment.get(consumer2)); }
Node node = metadata.fetch().nodes().get(0); PartitionAssignor assignor = new RangeAssignor();
@Test public void testTwoConsumersTwoTopicsSixPartitions() { String topic1 = "topic1"; String topic2 = "topic2"; String consumer1 = "consumer1"; String consumer2 = "consumer2"; Map<String, Integer> partitionsPerTopic = new HashMap<>(); partitionsPerTopic.put(topic1, 3); partitionsPerTopic.put(topic2, 3); Map<String, Subscription> consumers = new HashMap<>(); consumers.put(consumer1, new Subscription(topics(topic1, topic2))); consumers.put(consumer2, new Subscription(topics(topic1, topic2))); Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers); assertAssignment(partitions(tp(topic1, 0), tp(topic1, 1), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumer1)); assertAssignment(partitions(tp(topic1, 2), tp(topic2, 2)), assignment.get(consumer2)); }
private KafkaConsumer<String, String> newConsumer(Time time, KafkaClient client, Metadata metadata, PartitionAssignor assignor, OffsetResetStrategy resetStrategy, boolean autoCommitEnabled, String groupId) { String clientId = "mock-consumer"; String metricGroupPrefix = "consumer"; long retryBackoffMs = 100; int requestTimeoutMs = 30000; int defaultApiTimeoutMs = 30000; boolean excludeInternalTopics = true; int minBytes = 1; int maxBytes = Integer.MAX_VALUE; int maxWaitMs = 500; int fetchSize = 1024 * 1024; int maxPollRecords = Integer.MAX_VALUE; boolean checkCrcs = true; int rebalanceTimeoutMs = 60000; Deserializer<String> keyDeserializer = new StringDeserializer(); Deserializer<String> valueDeserializer = new StringDeserializer(); List<PartitionAssignor> assignors = singletonList(assignor); ConsumerInterceptors<String, String> interceptors = new ConsumerInterceptors<>(Collections.emptyList()); Metrics metrics = new Metrics(); ConsumerMetrics metricsRegistry = new ConsumerMetrics(metricGroupPrefix);