forked from apache/kafka
/
AbstractPartitionAssignor.java
132 lines (116 loc) · 5.55 KB
/
AbstractPartitionAssignor.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.clients.consumer.internals;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.TopicPartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
/**
* Abstract assignor implementation which does some common grunt work (in particular collecting
* partition counts which are always needed in assignors).
*/
public abstract class AbstractPartitionAssignor implements ConsumerPartitionAssignor {
private static final Logger log = LoggerFactory.getLogger(AbstractPartitionAssignor.class);
/**
* Perform the group assignment given the partition counts and member subscriptions
* @param partitionsPerTopic The number of partitions for each subscribed topic. Topics not in metadata will be excluded
* from this map.
* @param subscriptions Map from the member id to their respective topic subscription
* @return Map from each member to the list of partitions assigned to them.
*/
public abstract Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
Map<String, Subscription> subscriptions);
@Override
public GroupAssignment assign(Cluster metadata, GroupSubscription groupSubscriptions) {
Map<String, Subscription> subscriptions = groupSubscriptions.groupSubscription();
Set<String> allSubscribedTopics = new HashSet<>();
for (Map.Entry<String, Subscription> subscriptionEntry : subscriptions.entrySet())
allSubscribedTopics.addAll(subscriptionEntry.getValue().topics());
Map<String, Integer> partitionsPerTopic = new HashMap<>();
for (String topic : allSubscribedTopics) {
Integer numPartitions = metadata.partitionCountForTopic(topic);
if (numPartitions != null && numPartitions > 0)
partitionsPerTopic.put(topic, numPartitions);
else
log.debug("Skipping assignment for topic {} since no metadata is available", topic);
}
Map<String, List<TopicPartition>> rawAssignments = assign(partitionsPerTopic, subscriptions);
// this class maintains no user data, so just wrap the results
Map<String, Assignment> assignments = new HashMap<>();
for (Map.Entry<String, List<TopicPartition>> assignmentEntry : rawAssignments.entrySet())
assignments.put(assignmentEntry.getKey(), new Assignment(assignmentEntry.getValue()));
return new GroupAssignment(assignments);
}
protected static <K, V> void put(Map<K, List<V>> map, K key, V value) {
List<V> list = map.computeIfAbsent(key, k -> new ArrayList<>());
list.add(value);
}
protected static List<TopicPartition> partitions(String topic, int numPartitions) {
List<TopicPartition> partitions = new ArrayList<>(numPartitions);
for (int i = 0; i < numPartitions; i++)
partitions.add(new TopicPartition(topic, i));
return partitions;
}
public static class MemberInfo implements Comparable<MemberInfo> {
public final String memberId;
public final Optional<String> groupInstanceId;
public MemberInfo(String memberId, Optional<String> groupInstanceId) {
this.memberId = memberId;
this.groupInstanceId = groupInstanceId;
}
@Override
public int compareTo(MemberInfo otherMemberInfo) {
if (this.groupInstanceId.isPresent() &&
otherMemberInfo.groupInstanceId.isPresent()) {
return this.groupInstanceId.get()
.compareTo(otherMemberInfo.groupInstanceId.get());
} else if (this.groupInstanceId.isPresent()) {
return -1;
} else if (otherMemberInfo.groupInstanceId.isPresent()) {
return 1;
} else {
return this.memberId.compareTo(otherMemberInfo.memberId);
}
}
@Override
public boolean equals(Object o) {
return o instanceof MemberInfo && this.memberId.equals(((MemberInfo) o).memberId);
}
/**
* We could just use member.id to be the hashcode, since it's unique
* across the group.
*/
@Override
public int hashCode() {
return memberId.hashCode();
}
@Override
public String toString() {
return "MemberInfo [member.id: " + memberId
+ ", group.instance.id: " + groupInstanceId.orElse("{}")
+ "]";
}
}
}