-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-17747: [2/N] Add compute topic and group hash #19523
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 16 commits
36cb999
f4c9749
ced64d7
7e00a14
77b99da
aff55ce
7f7649a
8fc5b33
d2427f9
b5eebbf
b6ccef3
a1be1f6
9b4294f
478dd98
9fc8e8e
167cd4a
6e219a9
08cdd37
527b387
cde92d8
b238375
a1b658a
ed7b450
44c1337
f1ea74b
bb1edb3
3382702
c031a29
ec7024c
cf35b7a
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 |
|---|---|---|
|
|
@@ -23,28 +23,41 @@ | |
| import org.apache.kafka.common.message.ConsumerProtocolAssignment; | ||
| import org.apache.kafka.common.message.ConsumerProtocolSubscription; | ||
| import org.apache.kafka.common.protocol.ApiMessage; | ||
| import org.apache.kafka.common.utils.ByteBufferOutputStream; | ||
| import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; | ||
| import org.apache.kafka.coordinator.group.generated.ShareGroupCurrentMemberAssignmentValue; | ||
| import org.apache.kafka.image.ClusterImage; | ||
| import org.apache.kafka.image.TopicImage; | ||
| import org.apache.kafka.image.TopicsImage; | ||
| import org.apache.kafka.metadata.BrokerRegistration; | ||
| import org.apache.kafka.server.common.ApiMessageAndVersion; | ||
|
|
||
| import com.google.re2j.Pattern; | ||
| import com.google.re2j.PatternSyntaxException; | ||
|
|
||
| import net.jpountz.xxhash.XXHash64; | ||
| import net.jpountz.xxhash.XXHashFactory; | ||
|
|
||
| import java.io.DataOutputStream; | ||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.Collection; | ||
| import java.util.Collections; | ||
| import java.util.HashMap; | ||
| import java.util.HashSet; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Objects; | ||
| import java.util.Optional; | ||
| import java.util.OptionalInt; | ||
| import java.util.OptionalLong; | ||
| import java.util.Set; | ||
| import java.util.function.LongFunction; | ||
| import java.util.stream.Collectors; | ||
| import java.util.stream.IntStream; | ||
|
|
||
| public class Utils { | ||
| private Utils() {} | ||
|
|
@@ -324,4 +337,87 @@ static void throwIfRegularExpressionIsInvalid( | |
| regex, ex.getDescription())); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * The magic byte used to identify the version of topic hash function. | ||
| */ | ||
| static final byte TOPIC_HASH_MAGIC_BYTE = 0x00; | ||
| static final XXHash64 LZ4_HASH_INSTANCE = XXHashFactory.fastestInstance().hash64(); | ||
|
|
||
| /** | ||
| * 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) { | ||
|
Member
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. nit: public?
Member
Author
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. IIUC, this function will only be used in
Member
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. Ok. It works for me. |
||
| // Convert long to byte array. This is taken from guava LongHashCode#asBytes. | ||
| // https://github.com/google/guava/blob/bdf2a9d05342fca852645278d474082905e09d94/guava/src/com/google/common/hash/HashCode.java#L187-L199 | ||
| LongFunction<byte[]> longToBytes = (long value) -> new byte[] { | ||
| (byte) value, | ||
| (byte) (value >> 8), | ||
| (byte) (value >> 16), | ||
| (byte) (value >> 24), | ||
| (byte) (value >> 32), | ||
| (byte) (value >> 40), | ||
| (byte) (value >> 48), | ||
| (byte) (value >> 56) | ||
| }; | ||
|
|
||
| // Combine the sorted topic hashes. | ||
| byte[] resultBytes = new byte[8]; | ||
| topicHashes.entrySet() | ||
| .stream() | ||
| .sorted(Map.Entry.comparingByKey()) // sort by topic name | ||
| .map(Map.Entry::getValue) | ||
| .map(longToBytes::apply) | ||
| .forEach(nextBytes -> { | ||
|
||
| // Combine ordered hashes. This is taken from guava Hashing#combineOrdered. | ||
| // https://github.com/google/guava/blob/bdf2a9d05342fca852645278d474082905e09d94/guava/src/com/google/common/hash/Hashing.java#L689-L712 | ||
| for (int i = 0; i < nextBytes.length; i++) { | ||
| resultBytes[i] = (byte) (resultBytes[i] * 37 ^ nextBytes[i]); | ||
| } | ||
| }); | ||
|
|
||
| return LZ4_HASH_INSTANCE.hash(resultBytes, 0, resultBytes.length, 0); | ||
| } | ||
|
|
||
| /** | ||
| * Computes the hash of the topic id, name, number of partitions, and partition racks by XXHash64. | ||
| * | ||
| * @param topicImage The topic image. | ||
| * @param clusterImage The cluster image. | ||
| * @return The hash of the topic. | ||
| */ | ||
| static long computeTopicHash(TopicImage topicImage, ClusterImage clusterImage) throws IOException { | ||
|
||
| try (ByteBufferOutputStream bbos = new ByteBufferOutputStream(512); | ||
| DataOutputStream dos = new DataOutputStream(bbos)) { | ||
| dos.writeByte(TOPIC_HASH_MAGIC_BYTE); // magic byte | ||
| dos.writeLong(topicImage.id().hashCode()); // topic ID | ||
| dos.writeUTF(topicImage.name()); // topic name | ||
| dos.writeInt(topicImage.partitions().size()); // number of partitions | ||
| for (int i = 0; i < topicImage.partitions().size(); i++) { | ||
| dos.writeInt(i); // partition id | ||
| List<String> sortedRacksList = Arrays.stream(topicImage.partitions().get(i).replicas) | ||
| .mapToObj(clusterImage::broker) | ||
| .filter(Objects::nonNull) | ||
| .map(BrokerRegistration::rack) | ||
| .filter(Optional::isPresent) | ||
| .map(Optional::get) | ||
| .sorted() | ||
| .toList(); | ||
|
|
||
| String racks = IntStream.range(0, sortedRacksList.size()) | ||
| // The rack string combination cannot use simple separator like ",", because there is no limitation for rack character. | ||
| // If using simple separator like "," it may hit edge case like ",," and ",,," / ",,," and ",,". | ||
| // Add index before the rack string to avoid the edge case. | ||
| .mapToObj(idx -> idx + ":" + sortedRacksList.get(idx)) // Format: "index:value" | ||
| .collect(Collectors.joining(",")); // Separator between "index:value" pairs | ||
| dos.writeUTF(racks); // sorted racks | ||
| } | ||
| dos.flush(); | ||
| ByteBuffer topicBytes = bbos.buffer().flip(); | ||
| return LZ4_HASH_INSTANCE.hash(topicBytes, 0); | ||
|
||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,201 @@ | ||
| /* | ||
| * 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.common.utils.ByteBufferOutputStream; | ||
| import org.apache.kafka.image.MetadataImage; | ||
|
|
||
| import net.jpountz.xxhash.XXHash64; | ||
| import net.jpountz.xxhash.XXHashFactory; | ||
|
|
||
| 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.io.DataOutputStream; | ||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.LinkedHashMap; | ||
| 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 UtilsTest { | ||
| 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(); | ||
| private static final XXHash64 LZ4_HASH_INSTANCE = XXHashFactory.fastestInstance().hash64(); | ||
|
||
|
|
||
| @Test | ||
| void testComputeTopicHash() throws IOException { | ||
| long result = Utils.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
||
|
|
||
| try (ByteBufferOutputStream bbos = new ByteBufferOutputStream(512); | ||
| DataOutputStream dos = new DataOutputStream(bbos)) { | ||
| dos.writeByte(0); // magic byte | ||
| dos.writeLong(FOO_TOPIC_ID.hashCode()); // topic ID | ||
| dos.writeUTF(FOO_TOPIC_NAME); // topic name | ||
| dos.writeInt(FOO_NUM_PARTITIONS); // number of partitions | ||
| dos.writeInt(0); // partition 0 | ||
| dos.writeUTF("0:rack0,1:rack1"); // rack of partition 0 | ||
| dos.writeInt(1); // partition 1 | ||
| dos.writeUTF("0:rack1,1:rack2"); // rack of partition 1 | ||
| dos.flush(); | ||
| ByteBuffer topicBytes = bbos.buffer().flip(); | ||
| assertEquals(LZ4_HASH_INSTANCE.hash(topicBytes, 0), result); | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeTopicHashWithDifferentMagicByte() throws IOException { | ||
| long result = Utils.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| try (ByteBufferOutputStream bbos = new ByteBufferOutputStream(512); | ||
| DataOutputStream dos = new DataOutputStream(bbos)) { | ||
| dos.writeByte(1); // different magic byte | ||
| dos.writeLong(FOO_TOPIC_ID.hashCode()); // topic ID | ||
| dos.writeUTF(FOO_TOPIC_NAME); // topic name | ||
| dos.writeInt(FOO_NUM_PARTITIONS); // number of partitions | ||
| dos.writeInt(0); // partition 0 | ||
| dos.writeUTF("0:rack0,1:rack1"); // rack of partition 0 | ||
| dos.writeInt(1); // partition 1 | ||
| dos.writeUTF("0:rack1,1:rack2"); // rack of partition 1 | ||
| dos.flush(); | ||
| ByteBuffer topicBytes = bbos.buffer().flip(); | ||
| assertNotEquals(LZ4_HASH_INSTANCE.hash(topicBytes, 0), result); | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeTopicHashWithDifferentPartitionOrder() throws IOException { | ||
| long result = Utils.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| try (ByteBufferOutputStream bbos = new ByteBufferOutputStream(512); | ||
| DataOutputStream dos = new DataOutputStream(bbos)) { | ||
| dos.writeByte(0); // magic byte | ||
| dos.writeLong(FOO_TOPIC_ID.hashCode()); // topic ID | ||
| dos.writeUTF(FOO_TOPIC_NAME); // topic name | ||
| dos.writeInt(FOO_NUM_PARTITIONS); // number of partitions | ||
| // different partition order | ||
| dos.writeInt(1); // partition 1 | ||
| dos.writeUTF("0:rack1,1:rack2"); // rack of partition 1 | ||
| dos.writeInt(0); // partition 0 | ||
| dos.writeUTF("0:rack0,1:rack1"); // rack of partition 0 | ||
| dos.flush(); | ||
| ByteBuffer topicBytes = bbos.buffer().flip(); | ||
| assertNotEquals(LZ4_HASH_INSTANCE.hash(topicBytes, 0), result); | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeTopicHashWithDifferentRackOrder() throws IOException { | ||
| long result = Utils.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| try (ByteBufferOutputStream bbos = new ByteBufferOutputStream(512); | ||
| DataOutputStream dos = new DataOutputStream(bbos)) { | ||
| dos.writeByte(0); // magic byte | ||
| dos.writeLong(FOO_TOPIC_ID.hashCode()); // topic ID | ||
| dos.writeUTF(FOO_TOPIC_NAME); // topic name | ||
| dos.writeInt(FOO_NUM_PARTITIONS); // number of partitions | ||
| dos.writeInt(0); // partition 0 | ||
| dos.writeUTF("0:rack1,1:rack0"); // different rack order of partition 0 | ||
| dos.writeInt(1); // partition 1 | ||
| dos.writeUTF("0:rack1,1:rack2"); // rack of partition 1 | ||
| dos.flush(); | ||
| ByteBuffer topicBytes = bbos.buffer().flip(); | ||
| assertNotEquals(LZ4_HASH_INSTANCE.hash(topicBytes, 0), result); | ||
| } | ||
| } | ||
|
|
||
| @ParameterizedTest | ||
| @MethodSource("differentFieldGenerator") | ||
| void testComputeTopicHashWithDifferentField(MetadataImage differentImage, Uuid topicId) throws IOException { | ||
| long result = Utils.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster()); | ||
|
|
||
| assertNotEquals( | ||
| Utils.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 testComputeGroupHashWithDifferentOrder() { | ||
| Map<String, Long> ascendTopicHashes = new LinkedHashMap<>(); | ||
| ascendTopicHashes.put(BAR_TOPIC_NAME, 123L); | ||
| ascendTopicHashes.put(FOO_TOPIC_NAME, 456L); | ||
|
|
||
| Map<String, Long> descendTopicHashes = new LinkedHashMap<>(); | ||
| descendTopicHashes.put(FOO_TOPIC_NAME, 456L); | ||
| descendTopicHashes.put(BAR_TOPIC_NAME, 123L); | ||
| assertEquals(Utils.computeGroupHash(ascendTopicHashes), Utils.computeGroupHash(descendTopicHashes)); | ||
| } | ||
|
|
||
| @Test | ||
| void testComputeGroupHashWithSameKeyButDifferentValue() { | ||
| Map<String, Long> map1 = Map.of( | ||
| BAR_TOPIC_NAME, 123L, | ||
| FOO_TOPIC_NAME, 456L | ||
| ); | ||
|
|
||
| Map<String, Long> map2 = Map.of( | ||
| BAR_TOPIC_NAME, 456L, | ||
| FOO_TOPIC_NAME, 123L | ||
| ); | ||
| assertNotEquals(Utils.computeGroupHash(map1), Utils.computeGroupHash(map2)); | ||
| } | ||
| } | ||
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.
Do we need this change?
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.
Removed it. Thanks.