MINOR: Various cleanups in coordinator modules (#17828)

Reviewers: David Jacot <djacot@confluent.io>, Ken Huang <s7133700@gmail.com>
This commit is contained in:
Mickael Maison 2024-11-19 10:01:05 +01:00 committed by GitHub
parent 624cd4f7d0
commit 389f96aabd
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
20 changed files with 88 additions and 130 deletions

View File

@ -2333,7 +2333,7 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
if (context != null) {
context.lock.lock();
try {
if (!partitionEpoch.isPresent() || context.epoch < partitionEpoch.getAsInt()) {
if (partitionEpoch.isEmpty() || context.epoch < partitionEpoch.getAsInt()) {
log.info("Started unloading metadata for {} with epoch {}.", tp, partitionEpoch);
context.transitionTo(CoordinatorState.CLOSED);
coordinators.remove(tp, context);

View File

@ -37,7 +37,7 @@ import static org.junit.jupiter.api.Assertions.assertNull;
public class EventAccumulatorTest {
private class MockEvent implements EventAccumulator.Event<Integer> {
private static class MockEvent implements EventAccumulator.Event<Integer> {
int key;
int value;
@ -153,7 +153,7 @@ public class EventAccumulatorTest {
accumulator.addLast(event2);
assertEquals(3, accumulator.size());
MockEvent event = null;
MockEvent event;
// Poll event0.
event = accumulator.poll();

View File

@ -37,10 +37,10 @@ import java.util.stream.StreamSupport;
*/
public class InMemoryPartitionWriter implements PartitionWriter {
private class PartitionState {
private ReentrantLock lock = new ReentrantLock();
private List<Listener> listeners = new ArrayList<>();
private List<MemoryRecords> entries = new ArrayList<>();
private static class PartitionState {
private final ReentrantLock lock = new ReentrantLock();
private final List<Listener> listeners = new ArrayList<>();
private final List<MemoryRecords> entries = new ArrayList<>();
private long endOffset = 0L;
private long committedOffset = 0L;
}
@ -134,9 +134,8 @@ public class InMemoryPartitionWriter implements PartitionWriter {
state.lock.lock();
try {
state.committedOffset = offset;
state.listeners.forEach(listener -> {
listener.onHighWatermarkUpdated(tp, state.committedOffset);
});
state.listeners.forEach(listener ->
listener.onHighWatermarkUpdated(tp, state.committedOffset));
} finally {
state.lock.unlock();
}
@ -149,9 +148,8 @@ public class InMemoryPartitionWriter implements PartitionWriter {
state.lock.lock();
try {
state.committedOffset = state.endOffset;
state.listeners.forEach(listener -> {
listener.onHighWatermarkUpdated(tp, state.committedOffset);
});
state.listeners.forEach(listener ->
listener.onHighWatermarkUpdated(tp, state.committedOffset));
} finally {
state.lock.unlock();
}

View File

@ -420,7 +420,6 @@ public interface GroupCoordinator {
*
* @param groupId The group id.
* @param newGroupConfig The new group config
* @return void
*/
void updateGroupConfig(String groupId, Properties newGroupConfig);

View File

@ -78,8 +78,10 @@ public class GroupCoordinatorConfig {
public static final String GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC = "The list of enabled rebalance protocols. Supported protocols: " +
Arrays.stream(Group.GroupType.values()).map(Group.GroupType::toString).collect(Collectors.joining(",")) + ". " +
"The " + Group.GroupType.SHARE + " rebalance protocol is in early access and therefore must not be used in production.";
public static final List<String> GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT =
Collections.unmodifiableList(Arrays.asList(Group.GroupType.CLASSIC.toString(), Group.GroupType.CONSUMER.toString()));
public static final List<String> GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT = List.of(
Group.GroupType.CLASSIC.toString(),
Group.GroupType.CONSUMER.toString()
);
public static final String GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG = "group.coordinator.append.linger.ms";
public static final String GROUP_COORDINATOR_APPEND_LINGER_MS_DOC = "The duration in milliseconds that the coordinator will " +
"wait for writes to accumulate before flushing them to disk. Transactional writes are not accumulated.";
@ -120,10 +122,10 @@ public class GroupCoordinatorConfig {
public static final String CONSUMER_GROUP_ASSIGNORS_CONFIG = "group.consumer.assignors";
public static final String CONSUMER_GROUP_ASSIGNORS_DOC = "The server side assignors as a list of full class names. The first one in the list is considered as the default assignor to be used in the case where the consumer does not specify an assignor.";
public static final List<String> CONSUMER_GROUP_ASSIGNORS_DEFAULT = Collections.unmodifiableList(Arrays.asList(
UniformAssignor.class.getName(),
RangeAssignor.class.getName()
));
public static final List<String> CONSUMER_GROUP_ASSIGNORS_DEFAULT = List.of(
UniformAssignor.class.getName(),
RangeAssignor.class.getName()
);
public static final String CONSUMER_GROUP_MIGRATION_POLICY_CONFIG = "group.consumer.migration.policy";
public static final String CONSUMER_GROUP_MIGRATION_POLICY_DEFAULT = ConsumerGroupMigrationPolicy.BIDIRECTIONAL.toString();
@ -380,10 +382,9 @@ public class GroupCoordinatorConfig {
* Copy the subset of properties that are relevant to consumer group.
*/
public Map<String, Integer> extractConsumerGroupConfigMap() {
Map<String, Integer> groupProps = new HashMap<>();
groupProps.put(GroupConfig.CONSUMER_SESSION_TIMEOUT_MS_CONFIG, consumerGroupSessionTimeoutMs());
groupProps.put(GroupConfig.CONSUMER_HEARTBEAT_INTERVAL_MS_CONFIG, consumerGroupHeartbeatIntervalMs());
return Collections.unmodifiableMap(groupProps);
return Map.of(
GroupConfig.CONSUMER_SESSION_TIMEOUT_MS_CONFIG, consumerGroupSessionTimeoutMs(),
GroupConfig.CONSUMER_HEARTBEAT_INTERVAL_MS_CONFIG, consumerGroupHeartbeatIntervalMs());
}
/**

View File

@ -141,13 +141,13 @@ public class GroupCoordinatorRecordHelpers {
Map<String, TopicMetadata> newSubscriptionMetadata
) {
ConsumerGroupPartitionMetadataValue value = new ConsumerGroupPartitionMetadataValue();
newSubscriptionMetadata.forEach((topicName, topicMetadata) -> {
newSubscriptionMetadata.forEach((topicName, topicMetadata) ->
value.topics().add(new ConsumerGroupPartitionMetadataValue.TopicMetadata()
.setTopicId(topicMetadata.id())
.setTopicName(topicMetadata.name())
.setNumPartitions(topicMetadata.numPartitions())
);
});
)
);
return new CoordinatorRecord(
new ApiMessageAndVersion(
@ -674,13 +674,13 @@ public class GroupCoordinatorRecordHelpers {
Map<String, TopicMetadata> newSubscriptionMetadata
) {
ShareGroupPartitionMetadataValue value = new ShareGroupPartitionMetadataValue();
newSubscriptionMetadata.forEach((topicName, topicMetadata) -> {
newSubscriptionMetadata.forEach((topicName, topicMetadata) ->
value.topics().add(new ShareGroupPartitionMetadataValue.TopicMetadata()
.setTopicId(topicMetadata.id())
.setTopicName(topicMetadata.name())
.setNumPartitions(topicMetadata.numPartitions())
);
});
)
);
return new CoordinatorRecord(
new ApiMessageAndVersion(

View File

@ -116,11 +116,11 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
public static class Builder implements CoordinatorShardBuilder<GroupCoordinatorShard, CoordinatorRecord> {
private final GroupCoordinatorConfig config;
private final GroupConfigManager groupConfigManager;
private LogContext logContext;
private SnapshotRegistry snapshotRegistry;
private Time time;
private CoordinatorTimer<Void, CoordinatorRecord> timer;
private GroupConfigManager groupConfigManager;
private CoordinatorMetrics coordinatorMetrics;
private TopicPartition topicPartition;

View File

@ -731,7 +731,7 @@ public class GroupMetadataManager {
ClassicGroup group = classicGroup(groupId, committedOffset);
if (group.isInState(STABLE)) {
if (!group.protocolName().isPresent()) {
if (group.protocolName().isEmpty()) {
throw new IllegalStateException("Invalid null group protocol for stable group");
}
@ -751,7 +751,7 @@ public class GroupMetadataManager {
.setGroupState(group.stateAsString())
.setProtocolType(group.protocolType().orElse(""))
.setMembers(group.allMembers().stream()
.map(member -> member.describeNoMetadata())
.map(ClassicGroupMember::describeNoMetadata)
.collect(Collectors.toList())
)
);
@ -3737,9 +3737,9 @@ public class GroupMetadataManager {
if (value != null) {
Map<String, TopicMetadata> subscriptionMetadata = new HashMap<>();
value.topics().forEach(topicMetadata -> {
subscriptionMetadata.put(topicMetadata.topicName(), TopicMetadata.fromRecord(topicMetadata));
});
value.topics().forEach(topicMetadata ->
subscriptionMetadata.put(topicMetadata.topicName(), TopicMetadata.fromRecord(topicMetadata))
);
group.setSubscriptionMetadata(subscriptionMetadata);
} else {
group.setSubscriptionMetadata(Collections.emptyMap());
@ -3947,19 +3947,19 @@ public class GroupMetadataManager {
case DEAD:
break;
case PREPARING_REBALANCE:
classicGroup.allMembers().forEach(member -> {
classicGroup.allMembers().forEach(member ->
classicGroup.completeJoinFuture(member, new JoinGroupResponseData()
.setMemberId(member.memberId())
.setErrorCode(NOT_COORDINATOR.code()));
});
.setErrorCode(NOT_COORDINATOR.code()))
);
break;
case COMPLETING_REBALANCE:
case STABLE:
classicGroup.allMembers().forEach(member -> {
classicGroup.allMembers().forEach(member ->
classicGroup.completeSyncFuture(member, new SyncGroupResponseData()
.setErrorCode(NOT_COORDINATOR.code()));
});
.setErrorCode(NOT_COORDINATOR.code()))
);
}
break;
case SHARE:
@ -6086,7 +6086,7 @@ public class GroupMetadataManager {
if (isEmptyClassicGroup(group)) {
// Delete the classic group by adding tombstones.
// There's no need to remove the group as the replay of tombstones removes it.
if (group != null) createGroupTombstoneRecords(group, records);
createGroupTombstoneRecords(group, records);
return true;
}
return false;

View File

@ -872,7 +872,7 @@ public class OffsetMetadataManager {
long currentTimestampMs = time.milliseconds();
Optional<OffsetExpirationCondition> offsetExpirationCondition = group.offsetExpirationCondition();
if (!offsetExpirationCondition.isPresent()) {
if (offsetExpirationCondition.isEmpty()) {
return false;
}

View File

@ -63,7 +63,7 @@ class RangeSet implements Set<Integer> {
@Override
public Iterator<Integer> iterator() {
return new Iterator<Integer>() {
return new Iterator<>() {
private int current = from;
@Override

View File

@ -24,9 +24,6 @@ import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignorExceptio
import org.apache.kafka.coordinator.group.api.assignor.SubscribedTopicDescriber;
import org.apache.kafka.coordinator.group.modern.MemberAssignmentImpl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@ -55,7 +52,6 @@ import java.util.Set;
* Balance > Stickiness.
*/
public class UniformHeterogeneousAssignmentBuilder {
private static final Logger LOG = LoggerFactory.getLogger(UniformHeterogeneousAssignmentBuilder.class);
/**
* The maximum number of iterations to perform in the final iterative balancing phase.
@ -181,50 +177,44 @@ public class UniformHeterogeneousAssignmentBuilder {
}
}
this.topicComparator = new Comparator<Uuid>() {
@Override
public int compare(final Uuid topic1Id, final Uuid topic2Id) {
int topic1PartitionCount = subscribedTopicDescriber.numPartitions(topic1Id);
int topic2PartitionCount = subscribedTopicDescriber.numPartitions(topic2Id);
int topic1SubscriberCount = topicSubscribers.get(topic1Id).size();
int topic2SubscriberCount = topicSubscribers.get(topic2Id).size();
this.topicComparator = (topic1Id, topic2Id) -> {
int topic1PartitionCount = subscribedTopicDescriber.numPartitions(topic1Id);
int topic2PartitionCount = subscribedTopicDescriber.numPartitions(topic2Id);
int topic1SubscriberCount = topicSubscribers.get(topic1Id).size();
int topic2SubscriberCount = topicSubscribers.get(topic2Id).size();
// Order by partitions per subscriber, descending.
int order = Double.compare(
(double) topic2PartitionCount / topic2SubscriberCount,
(double) topic1PartitionCount / topic1SubscriberCount
);
// Order by partitions per subscriber, descending.
int order = Double.compare(
(double) topic2PartitionCount / topic2SubscriberCount,
(double) topic1PartitionCount / topic1SubscriberCount
);
// Then order by subscriber count, ascending.
if (order == 0) {
order = Integer.compare(topic1SubscriberCount, topic2SubscriberCount);
}
// Then order by topic id, ascending.
if (order == 0) {
order = topic1Id.compareTo(topic2Id);
}
return order;
// Then order by subscriber count, ascending.
if (order == 0) {
order = Integer.compare(topic1SubscriberCount, topic2SubscriberCount);
}
// Then order by topic id, ascending.
if (order == 0) {
order = topic1Id.compareTo(topic2Id);
}
return order;
};
this.memberComparator = new Comparator<Integer>() {
@Override
public int compare(final Integer memberIndex1, final Integer memberIndex2) {
// Order by number of assigned partitions, ascending.
int order = Integer.compare(
memberTargetAssignmentSizes[memberIndex1],
memberTargetAssignmentSizes[memberIndex2]
);
this.memberComparator = (memberIndex1, memberIndex2) -> {
// Order by number of assigned partitions, ascending.
int order = Integer.compare(
memberTargetAssignmentSizes[memberIndex1],
memberTargetAssignmentSizes[memberIndex2]
);
// Then order by member index, ascending.
if (order == 0) {
order = memberIndex1.compareTo(memberIndex2);
}
return order;
// Then order by member index, ascending.
if (order == 0) {
order = memberIndex1.compareTo(memberIndex2);
}
return order;
};
// Initialize partition owners for the target assignments.
@ -851,14 +841,6 @@ public class UniformHeterogeneousAssignmentBuilder {
addPartitionToTargetAssignment(topicId, partition, memberIndex);
}
/**
* @param memberIndex The member index.
* @return The current assignment size for the given member.
*/
private int targetAssignmentSize(int memberIndex) {
return memberTargetAssignmentSizes[memberIndex];
}
/**
* Assigns a partition to a member and updates the current assignment size.
*

View File

@ -287,7 +287,7 @@ public class ClassicGroup implements Group {
* @return True if the group is a simple group.
*/
public boolean isSimpleGroup() {
return !protocolType.isPresent() && isEmpty() && pendingJoinMembers.isEmpty();
return protocolType.isEmpty() && isEmpty() && pendingJoinMembers.isEmpty();
}
/**
@ -448,7 +448,7 @@ public class ClassicGroup implements Group {
throw new IllegalStateException("None of the member's protocols can be supported.");
}
if (!leaderId.isPresent()) {
if (leaderId.isEmpty()) {
leaderId = Optional.of(member.memberId());
}
@ -971,23 +971,6 @@ public class ClassicGroup implements Group {
return statesFilter.contains(state.toLowerCaseString());
}
/**
* Verify the member id is up to date for static members. Return true if both conditions met:
* 1. given member is a known static member to group
* 2. group stored member id doesn't match with given member id
*
* @param groupInstanceId the group instance id.
* @param memberId the member id.
* @return whether the static member is fenced based on the condition above.
*/
public boolean isStaticMemberFenced(
String groupInstanceId,
String memberId
) {
String existingMemberId = staticMemberId(groupInstanceId);
return existingMemberId != null && !existingMemberId.equals(memberId);
}
/**
* @return whether the group can rebalance.
*/
@ -1160,7 +1143,7 @@ public class ClassicGroup implements Group {
* @return the subscribed topics or Empty based on the condition above.
*/
public Optional<Set<String>> computeSubscribedTopics() {
if (!protocolType.isPresent()) {
if (protocolType.isEmpty()) {
return Optional.empty();
}
String type = protocolType.get();
@ -1378,7 +1361,7 @@ public class ClassicGroup implements Group {
ClassicGroupState.STABLE,
time,
consumerGroup.groupEpoch(),
Optional.ofNullable(ConsumerProtocol.PROTOCOL_TYPE),
Optional.of(ConsumerProtocol.PROTOCOL_TYPE),
Optional.empty(),
Optional.empty(),
Optional.of(time.milliseconds())

View File

@ -73,7 +73,7 @@ public class UnionSet<T> implements Set<T> {
@Override
public Iterator<T> iterator() {
return new Iterator<T>() {
return new Iterator<>() {
private final Iterator<T> largeSetIterator = largeSet.iterator();
private final Iterator<T> smallSetIterator = smallSet.iterator();
private T next = null;

View File

@ -253,7 +253,7 @@ public class ConsumerGroupMember extends ModernGroupMember {
/**
* The rebalance timeout provided by the member.
*/
private int rebalanceTimeoutMs;
private final int rebalanceTimeoutMs;
/**
* The subscription pattern configured by the member.
@ -360,7 +360,7 @@ public class ConsumerGroupMember extends ModernGroupMember {
*/
public Optional<Integer> classicProtocolSessionTimeout() {
if (useClassicProtocol()) {
return Optional.ofNullable(classicMemberMetadata.sessionTimeoutMs());
return Optional.of(classicMemberMetadata.sessionTimeoutMs());
} else {
return Optional.empty();
}

View File

@ -23,8 +23,6 @@ import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.group.GroupConfig;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
@ -179,8 +177,6 @@ public class ShareGroupConfig {
* Copy the subset of properties that are relevant to share group.
*/
public Map<String, Integer> extractShareGroupConfigMap() {
Map<String, Integer> groupProps = new HashMap<>();
groupProps.put(GroupConfig.SHARE_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupRecordLockDurationMs());
return Collections.unmodifiableMap(groupProps);
return Map.of(GroupConfig.SHARE_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupRecordLockDurationMs());
}
}

View File

@ -36,7 +36,6 @@ import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
@SuppressWarnings("deprecation")
public class GroupCoordinatorConfigTest {
private static final List<ConfigDef> GROUP_COORDINATOR_CONFIG_DEFS = Arrays.asList(
GroupCoordinatorConfig.GROUP_COORDINATOR_CONFIG_DEF,

View File

@ -27,7 +27,7 @@ import org.apache.kafka.image.MetadataProvenance;
import java.util.Arrays;
public class MetadataImageBuilder {
private MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
private final MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
public MetadataImageBuilder addTopic(
Uuid topicId,

View File

@ -102,11 +102,11 @@ public class OffsetMetadataManagerTest {
private final MockCoordinatorTimer<Void, CoordinatorRecord> timer = new MockCoordinatorTimer<>(time);
private final LogContext logContext = new LogContext();
private final SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
private final GroupCoordinatorMetricsShard metrics = mock(GroupCoordinatorMetricsShard.class);
private final GroupConfigManager configManager = mock(GroupConfigManager.class);
private GroupMetadataManager groupMetadataManager = null;
private MetadataImage metadataImage = null;
private GroupCoordinatorConfig config = null;
private GroupCoordinatorMetricsShard metrics = mock(GroupCoordinatorMetricsShard.class);
private GroupConfigManager configManager = mock(GroupConfigManager.class);
Builder withOffsetMetadataMaxSize(int offsetMetadataMaxSize) {
config = GroupCoordinatorConfigTest.createGroupCoordinatorConfig(offsetMetadataMaxSize, 60000L, 24 * 60);

View File

@ -45,8 +45,7 @@ public class UnionSetTest {
Set.of(2, 3, 4, 5)
);
List<Integer> result = new ArrayList<>();
result.addAll(union);
List<Integer> result = new ArrayList<>(union);
result.sort(Integer::compareTo);
assertEquals(List.of(1, 2, 3, 4, 5), result);

View File

@ -27,6 +27,7 @@ import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
@ -88,7 +89,7 @@ class TransactionLogConfigTest {
TransactionLogConfig transactionLogConfig = new TransactionLogConfig(config);
assertEquals(false, transactionLogConfig.transactionPartitionVerificationEnable());
assertFalse(transactionLogConfig.transactionPartitionVerificationEnable());
assertEquals(88, transactionLogConfig.producerIdExpirationMs());
// If the following calls are missing, we wont be able to distinguish whether the value is set in the constructor or if