-
Notifications
You must be signed in to change notification settings - Fork 14.4k
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
base: trunk
Are you sure you want to change the base?
Changes from all commits
36cb999
f4c9749
ced64d7
7e00a14
77b99da
aff55ce
7f7649a
8fc5b33
d2427f9
b5eebbf
b6ccef3
a1be1f6
9b4294f
478dd98
9fc8e8e
167cd4a
6e219a9
08cdd37
527b387
cde92d8
b238375
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,15 +23,24 @@ | |
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.Collection; | ||
import java.util.Collections; | ||
|
@@ -44,6 +53,7 @@ | |
import java.util.OptionalInt; | ||
import java.util.OptionalLong; | ||
import java.util.Set; | ||
import java.util.function.LongFunction; | ||
import java.util.stream.Collectors; | ||
|
||
public class Utils { | ||
|
@@ -324,4 +334,112 @@ 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. | ||
* <p> | ||
* The computed hash value is stored as part of the metadata hash in the *GroupMetadataValue. | ||
* <p> | ||
* The hashing process involves the following steps: | ||
* 1. Sort the topic hashes by topic name. | ||
* 2. Convert each long hash value into a byte array. | ||
* 3. Combine the sorted byte arrays to produce a final hash for the 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) { | ||
// 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]; | ||
|
||
// Sort entries by topic name | ||
List<Map.Entry<String, Long>> sortedEntries = new ArrayList<>(topicHashes.entrySet()); | ||
sortedEntries.sort(Map.Entry.comparingByKey()); | ||
|
||
for (Map.Entry<String, Long> entry : sortedEntries) { | ||
Long value = entry.getValue(); | ||
byte[] nextBytes = longToBytes.apply(value); | ||
|
||
// 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. | ||
* <p> | ||
* The computed hash value for the topic is utilized in conjunction with the {@link #computeGroupHash(Map)} | ||
* method and is stored as part of the metadata hash in the *GroupMetadataValue. | ||
* It is important to note that if the hash algorithm is changed, the magic byte must be updated to reflect the | ||
* new hash version. | ||
* <p> | ||
* The hashing process involves the following steps: | ||
* 1. Write a magic byte to denote the version of the hash function. | ||
* 2. Write the hash code of the topic ID. | ||
* 3. Write the UTF-8 encoded topic name. | ||
* 4. Write the number of partitions associated with the topic. | ||
* 5. For each partition, write the partition ID and a sorted list of rack identifiers. | ||
* - Rack identifiers are formatted as "<length1><value1><length2><value2>" to prevent issues with simple separators. | ||
* | ||
* @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 | ||
// 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 length before the rack string to avoid the edge case. | ||
List<String> racks = new ArrayList<>(); | ||
for (int replicaId : topicImage.partitions().get(i).replicas) { | ||
BrokerRegistration broker = clusterImage.broker(replicaId); | ||
if (broker != null) { | ||
Optional<String> rackOptional = broker.rack(); | ||
rackOptional.ifPresent(racks::add); | ||
} | ||
} | ||
|
||
Collections.sort(racks); | ||
for (String rack : racks) { | ||
// Format: "<length><value>" | ||
dos.writeInt(rack.length()); | ||
dos.writeUTF(rack); | ||
} | ||
} | ||
dos.flush(); | ||
ByteBuffer topicBytes = bbos.buffer().flip(); | ||
return LZ4_HASH_INSTANCE.hash(topicBytes, 0); | ||
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. There is also a streaming hash class - would that be a better option instead of creating the complete byte buffer? 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. @FrankYang0529 Have you had a chance to look into the streaming hash class? 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. Sorry, I just saw your comment here: #19523 (comment) |
||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,226 @@ | ||
/* | ||
* 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(); | ||
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.
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. Thanks for the suggestion. I do benchmark for streaming XXH3 / streaming XXH64 / non-streaming XXH3 / non-streaming XXH64. The streaming XXH3 gets the best result. However, it needs to include new library
TopicHashBenchmark.javapackage org.apache.kafka.jmh.metadata;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.image.ClusterDelta;
import org.apache.kafka.image.ClusterImage;
import org.apache.kafka.image.TopicsDelta;
import org.apache.kafka.image.TopicImage;
import org.apache.kafka.metadata.BrokerRegistration;
import com.dynatrace.hash4j.hashing.HashStream64;
import com.dynatrace.hash4j.hashing.Hashing;
import net.jpountz.xxhash.StreamingXXHash64;
import net.jpountz.xxhash.XXHashFactory;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import static org.apache.kafka.jmh.metadata.TopicsImageSnapshotLoadBenchmark.getInitialTopicsDelta;
import static org.apache.kafka.jmh.metadata.TopicsImageSnapshotLoadBenchmark.getNumBrokers;
@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 3)
@Measurement(iterations = 5)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.NANOSECONDS)
public class TopicHashBenchmark {
@Param({"10", "50", "100"})
private int partitionsPerTopic;
@Param({"3"})
private int replicationFactor;
@Param({"10"})
private int numReplicasPerBroker;
private TopicImage topicImage;
private ClusterImage clusterImage;
@Setup(Level.Trial)
public void setup() throws IOException {
TopicsDelta topicsDelta = getInitialTopicsDelta(1, partitionsPerTopic, replicationFactor, numReplicasPerBroker);
int numBrokers = getNumBrokers(1, partitionsPerTopic, replicationFactor, numReplicasPerBroker);
ClusterDelta clusterDelta = new ClusterDelta(ClusterImage.EMPTY);
for (int i = 0; i < numBrokers; i++) {
clusterDelta.replay(new RegisterBrokerRecord()
.setBrokerId(i)
.setRack(Uuid.randomUuid().toString())
);
}
topicImage = topicsDelta.apply().topicsById().values().stream().findFirst().get();
clusterImage = clusterDelta.apply();
}
@Benchmark
public void testLz4StreamingXXHash64() {
try (StreamingXXHash64 hash = XXHashFactory.fastestInstance().newStreamingHash64(0)) {
hash.update(new byte[]{(byte) 0}, 0, 1); // magic byte
// topic id
hash.update(intToBytes(topicImage.id().hashCode()), 0, 32);
// topic name
byte[] topicNameBytes = topicImage.name().getBytes();
hash.update(topicNameBytes, 0, topicNameBytes.length);
// number of partitions
hash.update(intToBytes(topicImage.partitions().size()), 0, 32);
for (int i = 0; i < topicImage.partitions().size(); i++) {
// partition id
hash.update(intToBytes(i), 0, 32);
// sorted racks
List<String> racks = new ArrayList<String>();
for (int replicaId : topicImage.partitions().get(i).replicas) {
BrokerRegistration broker = clusterImage.broker(replicaId);
if (broker != null) {
Optional<String> rackOptional = broker.rack();
rackOptional.ifPresent(racks::add);
}
}
Collections.sort(racks);
for (String rack : racks) {
// Format: "<length><value>"
byte[] rackBytes = rack.getBytes();
hash.update(intToBytes(rack.length()), 0, 32);
hash.update(rackBytes, 0, rackBytes.length);
}
}
hash.getValue();
}
}
@Benchmark
public void testLz4XXHash64() throws IOException {
try (ByteBufferOutputStream bbos = new ByteBufferOutputStream(512);
DataOutputStream dos = new DataOutputStream(bbos)) {
dos.writeByte(0); // 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
// 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 length before the rack string to avoid the edge case.
List<String> racks = new ArrayList<>();
for (int replicaId : topicImage.partitions().get(i).replicas) {
BrokerRegistration broker = clusterImage.broker(replicaId);
if (broker != null) {
Optional<String> rackOptional = broker.rack();
rackOptional.ifPresent(racks::add);
}
}
Collections.sort(racks);
for (String rack : racks) {
// Format: "<length><value>"
dos.writeInt(rack.length());
dos.writeUTF(rack);
}
}
dos.flush();
ByteBuffer topicBytes = bbos.buffer().flip();
XXHashFactory.fastestInstance().hash64().hash(topicBytes, 0);
}
}
@Benchmark
public void testDynatraceStreamingXXH3() {
HashStream64 hash = Hashing.xxh3_64().hashStream();
hash = hash.putByte((byte) 0)
.putLong(topicImage.id().hashCode())
.putString(topicImage.name())
.putInt(topicImage.partitions().size());
for (int i = 0; i < topicImage.partitions().size(); i++) {
// partition id
hash = hash.putInt(i);
// sorted racks
List<String> racks = new ArrayList<String>();
for (int replicaId : topicImage.partitions().get(i).replicas) {
BrokerRegistration broker = clusterImage.broker(replicaId);
if (broker != null) {
Optional<String> rackOptional = broker.rack();
rackOptional.ifPresent(racks::add);
}
}
Collections.sort(racks);
for (String rack : racks) {
// Format: "<length><value>"
hash.putInt(rack.length());
hash.putString(rack);
}
}
hash.getAsLong();
}
@Benchmark
public void testDynatraceXXH3() throws IOException {
try (ByteBufferOutputStream bbos = new ByteBufferOutputStream(512);
DataOutputStream dos = new DataOutputStream(bbos)) {
dos.writeByte(0); // 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
// 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 length before the rack string to avoid the edge case.
List<String> racks = new ArrayList<>();
for (int replicaId : topicImage.partitions().get(i).replicas) {
BrokerRegistration broker = clusterImage.broker(replicaId);
if (broker != null) {
Optional<String> rackOptional = broker.rack();
rackOptional.ifPresent(racks::add);
}
}
Collections.sort(racks);
for (String rack : racks) {
// Format: "<length><value>"
dos.writeInt(rack.length());
dos.writeUTF(rack);
}
}
dos.flush();
ByteBuffer topicBytes = bbos.buffer().flip();
Hashing.xxh3_64().hashBytesToLong(topicBytes.array());
}
}
private byte[] intToBytes(int value) {
return new byte[] {
(byte)(value >>> 24),
(byte)(value >>> 16),
(byte)(value >>> 8),
(byte)value
};
}
} 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 streaming XXH3 seems pretty interesting to me given the results. Is 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. From https://xxhash.com/, there are three Java libraries. Only zero-allocation-hashing and hash4j provides XXH3. However, only hash4j has streaming hash class. 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. |
||
|
||
@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.writeInt(5); // length of rack0 | ||
dos.writeUTF("rack0"); // The first rack in partition 0 | ||
dos.writeInt(5); // length of rack1 | ||
dos.writeUTF("rack1"); // The second rack in partition 0 | ||
dos.writeInt(1); // partition 1 | ||
dos.writeInt(5); // length of rack0 | ||
dos.writeUTF("rack1"); // The first rack in partition 1 | ||
dos.writeInt(5); // length of rack1 | ||
dos.writeUTF("rack2"); // The second rack in 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.writeInt(5); // length of rack0 | ||
dos.writeUTF("rack0"); // The first rack in partition 0 | ||
dos.writeInt(5); // length of rack1 | ||
dos.writeUTF("rack1"); // The second rack in partition 0 | ||
dos.writeInt(1); // partition 1 | ||
dos.writeInt(5); // length of rack0 | ||
dos.writeUTF("rack1"); // The first rack in partition 1 | ||
dos.writeInt(5); // length of rack1 | ||
dos.writeUTF("rack2"); // The second rack in 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.writeInt(5); // length of rack0 | ||
dos.writeUTF("rack1"); // The first rack in partition 1 | ||
dos.writeInt(5); // length of rack1 | ||
dos.writeUTF("rack2"); // The second rack in partition 1 | ||
dos.writeInt(0); // partition 0 | ||
dos.writeInt(5); // length of rack0 | ||
dos.writeUTF("rack0"); // The first rack in partition 0 | ||
dos.writeInt(5); // length of rack1 | ||
dos.writeUTF("rack1"); // The second rack in 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 | ||
// different rack order of partition 0 | ||
dos.writeInt(5); // length of rack0 | ||
dos.writeUTF("rack1"); // The second rack in partition 0 | ||
dos.writeInt(5); // length of rack1 | ||
dos.writeUTF("rack0"); // The first rack in partition 0 | ||
dos.writeInt(1); // partition 1 | ||
dos.writeInt(5); // length of rack0 | ||
dos.writeUTF("rack1"); // The first rack in partition 1 | ||
dos.writeInt(5); // length of rack1 | ||
dos.writeUTF("rack2"); // The second rack in 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.
Please add documentation to remind developers that the hash is stored as part of the state. Changing the implementation of the hashing function may break compatibility with existing states.
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.
If the hashing function is ever changed, is there a version field that should be updated?
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.
yes, there is a magic byte as version.