-
Notifications
You must be signed in to change notification settings - Fork 0
KAFKA-17747: Add compute topic and group hash #53
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -19,11 +19,21 @@ | |||||||||||||||||||||||||||||||||
| import org.apache.kafka.common.KafkaException; | ||||||||||||||||||||||||||||||||||
| import org.apache.kafka.common.message.ListGroupsResponseData; | ||||||||||||||||||||||||||||||||||
| import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord; | ||||||||||||||||||||||||||||||||||
| import org.apache.kafka.image.ClusterImage; | ||||||||||||||||||||||||||||||||||
| import org.apache.kafka.image.TopicImage; | ||||||||||||||||||||||||||||||||||
| import org.apache.kafka.metadata.BrokerRegistration; | ||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| import com.google.common.hash.HashCode; | ||||||||||||||||||||||||||||||||||
| import com.google.common.hash.HashFunction; | ||||||||||||||||||||||||||||||||||
| import com.google.common.hash.Hasher; | ||||||||||||||||||||||||||||||||||
| import com.google.common.hash.Hashing; | ||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| import java.nio.charset.StandardCharsets; | ||||||||||||||||||||||||||||||||||
| import java.util.Arrays; | ||||||||||||||||||||||||||||||||||
| import java.util.List; | ||||||||||||||||||||||||||||||||||
| import java.util.Locale; | ||||||||||||||||||||||||||||||||||
| import java.util.Map; | ||||||||||||||||||||||||||||||||||
| import java.util.Objects; | ||||||||||||||||||||||||||||||||||
| import java.util.Optional; | ||||||||||||||||||||||||||||||||||
| import java.util.Set; | ||||||||||||||||||||||||||||||||||
| import java.util.function.Function; | ||||||||||||||||||||||||||||||||||
|
|
@@ -209,4 +219,50 @@ void validateOffsetFetch( | |||||||||||||||||||||||||||||||||
| default boolean shouldExpire() { | ||||||||||||||||||||||||||||||||||
| return true; | ||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| /** | ||||||||||||||||||||||||||||||||||
| * Computes the hash of the topics in a group. | ||||||||||||||||||||||||||||||||||
| * | ||||||||||||||||||||||||||||||||||
| * @param topicHashes The map of topic hashes. Key is topic name and value is the topic hash. | ||||||||||||||||||||||||||||||||||
| * @return The hash of the group. | ||||||||||||||||||||||||||||||||||
| */ | ||||||||||||||||||||||||||||||||||
| static long computeGroupHash(Map<String, Long> topicHashes) { | ||||||||||||||||||||||||||||||||||
| return Hashing.combineOrdered( | ||||||||||||||||||||||||||||||||||
| topicHashes.entrySet() | ||||||||||||||||||||||||||||||||||
| .stream() | ||||||||||||||||||||||||||||||||||
| .sorted(Map.Entry.comparingByKey()) | ||||||||||||||||||||||||||||||||||
| .map(e -> HashCode.fromLong(e.getValue())) | ||||||||||||||||||||||||||||||||||
| .toList() | ||||||||||||||||||||||||||||||||||
| ).asLong(); | ||||||||||||||||||||||||||||||||||
|
Comment on lines
+230
to
+236
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Suggestion: When the topic hash map is empty, calling Severity Level: Minor
Suggested change
Why it matters? ⭐The suggestion is correct: Guava's Hashing.combineOrdered throws if given an empty iterable, so the current implementation can throw at runtime for an empty topicHashes map. The proposed change makes the behavior deterministic for that edge case and prevents a potential IllegalArgumentException. This is a real bug fix, not a mere stylistic change. Prompt for AI Agent 🤖This is a comment left during a code review.
**Path:** group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java
**Line:** 230:236
**Comment:**
*Logic Error: When the topic hash map is empty, calling `Hashing.combineOrdered` with an empty list will throw an IllegalArgumentException at runtime, so the group hash computation must handle the empty-case explicitly and return a deterministic value instead of delegating to `combineOrdered`.
Validate the correctness of the flagged issue. If correct, How can I resolve this? If you propose a fix, implement it and please make it concise. |
||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| /** | ||||||||||||||||||||||||||||||||||
| * Computes the hash of the topic id, name, number of partitions, and partition racks by Murmur3. | ||||||||||||||||||||||||||||||||||
| * | ||||||||||||||||||||||||||||||||||
| * @param topicImage The topic image. | ||||||||||||||||||||||||||||||||||
| * @param clusterImage The cluster image. | ||||||||||||||||||||||||||||||||||
| * @return The hash of the topic. | ||||||||||||||||||||||||||||||||||
| */ | ||||||||||||||||||||||||||||||||||
| static long computeTopicHash(TopicImage topicImage, ClusterImage clusterImage) { | ||||||||||||||||||||||||||||||||||
| HashFunction hf = Hashing.murmur3_128(); | ||||||||||||||||||||||||||||||||||
| Hasher topicHasher = hf.newHasher() | ||||||||||||||||||||||||||||||||||
| .putByte((byte) 0) // magic byte | ||||||||||||||||||||||||||||||||||
| .putLong(topicImage.id().hashCode()) // topic Id | ||||||||||||||||||||||||||||||||||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Using Consider changing the hasher chain to include both parts of the UUID: Hasher topicHasher = hf.newHasher()
.putByte((byte) 0) // magic byte
.putLong(topicImage.id().getMostSignificantBits())
.putLong(topicImage.id().getLeastSignificantBits())
.putString(topicImage.name(), StandardCharsets.UTF_8) // topic name
.putInt(topicImage.partitions().size()); // number of partitionsThis will also require updating the corresponding tests in There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. UUID Hashing WeaknessThe code uses topicImage.id().hashCode(), which converts a 128-bit UUID to a 32-bit integer before hashing. This discards 96 bits of information, significantly increasing the probability of hash collisions for different topic UUIDs, potentially causing incorrect topic state detection. Commitable Suggestion
Suggested change
Standards
|
||||||||||||||||||||||||||||||||||
| .putString(topicImage.name(), StandardCharsets.UTF_8) // topic name | ||||||||||||||||||||||||||||||||||
| .putInt(topicImage.partitions().size()); // number of partitions | ||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| topicImage.partitions().entrySet().stream().sorted(Map.Entry.comparingByKey()).forEach(entry -> { | ||||||||||||||||||||||||||||||||||
| topicHasher.putInt(entry.getKey()); // partition id | ||||||||||||||||||||||||||||||||||
| String racks = Arrays.stream(entry.getValue().replicas) | ||||||||||||||||||||||||||||||||||
| .mapToObj(clusterImage::broker) | ||||||||||||||||||||||||||||||||||
| .filter(Objects::nonNull) | ||||||||||||||||||||||||||||||||||
| .map(BrokerRegistration::rack) | ||||||||||||||||||||||||||||||||||
| .filter(Optional::isPresent) | ||||||||||||||||||||||||||||||||||
| .map(Optional::get) | ||||||||||||||||||||||||||||||||||
|
Comment on lines
+260
to
+261
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||||||||||||||||||||||||||||||||||
| .sorted() | ||||||||||||||||||||||||||||||||||
| .collect(Collectors.joining(";")); | ||||||||||||||||||||||||||||||||||
| topicHasher.putString(racks, StandardCharsets.UTF_8); // sorted racks with separator ";" | ||||||||||||||||||||||||||||||||||
| }); | ||||||||||||||||||||||||||||||||||
| return topicHasher.hash().asLong(); | ||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,189 @@ | ||
| /* | ||
| * 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.coordinator.group; | ||
|
|
||
| import org.apache.kafka.common.Uuid; | ||
| import org.apache.kafka.image.MetadataImage; | ||
|
|
||
| import com.google.common.hash.HashCode; | ||
| import com.google.common.hash.HashFunction; | ||
| import com.google.common.hash.Hasher; | ||
| import com.google.common.hash.Hashing; | ||
|
|
||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.params.ParameterizedTest; | ||
| import org.junit.jupiter.params.provider.Arguments; | ||
| import org.junit.jupiter.params.provider.MethodSource; | ||
|
|
||
| import java.nio.charset.StandardCharsets; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.stream.Stream; | ||
|
|
||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||
|
|
||
| public class GroupTest { | ||
| private static final Uuid FOO_TOPIC_ID = Uuid.randomUuid(); | ||
| private static final String FOO_TOPIC_NAME = "foo"; | ||
| private static final String BAR_TOPIC_NAME = "bar"; | ||
| private static final int FOO_NUM_PARTITIONS = 2; | ||
| private static final MetadataImage FOO_METADATA_IMAGE = new MetadataImageBuilder() | ||
| .addTopic(FOO_TOPIC_ID, FOO_TOPIC_NAME, FOO_NUM_PARTITIONS) | ||
| .addRacks() | ||
| .build(); | ||
|
|
||
| @Test | ||
| void testComputeTopicHash() { | ||
| long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| HashFunction hf = Hashing.murmur3_128(); | ||
| Hasher topicHasher = hf.newHasher() | ||
| .putByte((byte) 0) // magic byte | ||
| .putLong(FOO_TOPIC_ID.hashCode()) // topic Id | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Following the recommended change in Instead of .putLong(FOO_TOPIC_ID.getMostSignificantBits())
.putLong(FOO_TOPIC_ID.getLeastSignificantBits()) |
||
| .putString(FOO_TOPIC_NAME, StandardCharsets.UTF_8) // topic name | ||
| .putInt(FOO_NUM_PARTITIONS) // number of partitions | ||
| .putInt(0) // partition 0 | ||
| .putString("rack0;rack1", StandardCharsets.UTF_8) // rack of partition 0 | ||
| .putInt(1) // partition 1 | ||
| .putString("rack1;rack2", StandardCharsets.UTF_8); // rack of partition 1 | ||
| assertEquals(topicHasher.hash().asLong(), result); | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeTopicHashWithDifferentMagicByte() { | ||
| long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| HashFunction hf = Hashing.murmur3_128(); | ||
| Hasher topicHasher = hf.newHasher() | ||
| .putByte((byte) 1) // different magic byte | ||
| .putLong(FOO_TOPIC_ID.hashCode()) // topic Id | ||
| .putString(FOO_TOPIC_NAME, StandardCharsets.UTF_8) // topic name | ||
| .putInt(FOO_NUM_PARTITIONS) // number of partitions | ||
| .putInt(0) // partition 0 | ||
| .putString("rack0;rack1", StandardCharsets.UTF_8) // rack of partition 0 | ||
| .putInt(1) // partition 1 | ||
| .putString("rack1;rack2", StandardCharsets.UTF_8); // rack of partition 1 | ||
| assertNotEquals(topicHasher.hash().asLong(), result); | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeTopicHashWithDifferentPartitionOrder() { | ||
| long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| HashFunction hf = Hashing.murmur3_128(); | ||
| Hasher topicHasher = hf.newHasher() | ||
| .putByte((byte) 0) // magic byte | ||
| .putLong(FOO_TOPIC_ID.hashCode()) // topic Id | ||
| .putString(FOO_TOPIC_NAME, StandardCharsets.UTF_8) // topic name | ||
| .putInt(FOO_NUM_PARTITIONS) // number of partitions | ||
| // different partition order | ||
| .putInt(1) // partition 1 | ||
| .putString("rack1;rack2", StandardCharsets.UTF_8) // rack of partition 1 | ||
| .putInt(0) // partition 0 | ||
| .putString("rack0;rack1", StandardCharsets.UTF_8); // rack of partition 0 | ||
| assertNotEquals(topicHasher.hash().asLong(), result); | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeTopicHashWithDifferentRackOrder() { | ||
| long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| HashFunction hf = Hashing.murmur3_128(); | ||
| Hasher topicHasher = hf.newHasher() | ||
| .putByte((byte) 0) // magic byte | ||
| .putLong(FOO_TOPIC_ID.hashCode()) // topic Id | ||
| .putString(FOO_TOPIC_NAME, StandardCharsets.UTF_8) // topic name | ||
| .putInt(FOO_NUM_PARTITIONS) // number of partitions | ||
| .putInt(0) // partition 0 | ||
| .putString("rack1;rack0", StandardCharsets.UTF_8) // different rack order of partition 0 | ||
| .putInt(1) // partition 1 | ||
| .putString("rack1;rack2", StandardCharsets.UTF_8); // rack of partition 1 | ||
| assertNotEquals(topicHasher.hash().asLong(), result); | ||
| } | ||
|
|
||
| @ParameterizedTest | ||
| @MethodSource("differentFieldGenerator") | ||
| void testComputeTopicHashWithDifferentField(MetadataImage differentImage, Uuid topicId) { | ||
| long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| assertNotEquals( | ||
| Group.computeTopicHash( | ||
| differentImage.topics().getTopic(topicId), | ||
| differentImage.cluster() | ||
| ), | ||
| result | ||
| ); | ||
| } | ||
|
|
||
| private static Stream<Arguments> differentFieldGenerator() { | ||
| Uuid differentTopicId = Uuid.randomUuid(); | ||
| return Stream.of( | ||
| Arguments.of(new MetadataImageBuilder() // different topic id | ||
| .addTopic(differentTopicId, FOO_TOPIC_NAME, FOO_NUM_PARTITIONS) | ||
| .addRacks() | ||
| .build(), | ||
| differentTopicId | ||
| ), | ||
| Arguments.of(new MetadataImageBuilder() // different topic name | ||
| .addTopic(FOO_TOPIC_ID, "bar", FOO_NUM_PARTITIONS) | ||
| .addRacks() | ||
| .build(), | ||
| FOO_TOPIC_ID | ||
| ), | ||
| Arguments.of(new MetadataImageBuilder() // different partitions | ||
| .addTopic(FOO_TOPIC_ID, FOO_TOPIC_NAME, 1) | ||
| .addRacks() | ||
| .build(), | ||
| FOO_TOPIC_ID | ||
| ), | ||
| Arguments.of(new MetadataImageBuilder() // different racks | ||
| .addTopic(FOO_TOPIC_ID, FOO_TOPIC_NAME, FOO_NUM_PARTITIONS) | ||
| .build(), | ||
| FOO_TOPIC_ID | ||
| ) | ||
| ); | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeGroupHash() { | ||
| long result = Group.computeGroupHash(Map.of( | ||
| BAR_TOPIC_NAME, 123L, | ||
| FOO_TOPIC_NAME, 456L | ||
| )); | ||
|
|
||
| long expected = Hashing.combineOrdered(List.of( | ||
| HashCode.fromLong(123L), | ||
| HashCode.fromLong(456L) | ||
| )).asLong(); | ||
| assertEquals(expected, result); | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeGroupHashWithDifferentOrder() { | ||
| long result = Group.computeGroupHash(Map.of( | ||
| BAR_TOPIC_NAME, 123L, | ||
| FOO_TOPIC_NAME, 456L | ||
| )); | ||
|
|
||
| long unexpected = Hashing.combineOrdered(List.of( | ||
| HashCode.fromLong(456L), | ||
| HashCode.fromLong(123L) | ||
| )).asLong(); | ||
| assertNotEquals(unexpected, result); | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
🧩 Analysis chain
🏁 Script executed:
Repository: coderabbit-test/kafka
Length of output: 1247
Add Guava license entry to LICENSE-binary
Guava 33.4.0-jre is wired correctly in
gradle/dependencies.gradleand is already in use (build.gradle:1423). However, per the dependency file comment requiring LICENSE-binary updates for new third-party dependencies, Guava is missing from LICENSE-binary. Add an entry with the correct Apache 2.0 license text for Guava 33.4.0-jre.🤖 Prompt for AI Agents