Skip to content
Merged
Show file tree
Hide file tree
Changes from 12 commits
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
36cb999
KAFKA-17747: Add compute topic and group hash
FrankYang0529 Apr 21, 2025
f4c9749
KAFKA-17747: add guava license
FrankYang0529 Apr 21, 2025
ced64d7
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 Apr 21, 2025
7e00a14
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 Apr 28, 2025
77b99da
Revert "KAFKA-17747: add guava license"
FrankYang0529 Apr 28, 2025
aff55ce
KAFKA-17747: Move Murmur3 to org.apache.kafka.common.internals
FrankYang0529 Apr 28, 2025
7f7649a
KAFKA-17747: Replace guava with Murmur3
FrankYang0529 Apr 28, 2025
8fc5b33
remove unused package in import-control-group-coordinator.xml
FrankYang0529 Apr 28, 2025
d2427f9
remove sbt.json
FrankYang0529 Apr 28, 2025
b5eebbf
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 Apr 29, 2025
b6ccef3
Revert "KAFKA-17747: Move Murmur3 to org.apache.kafka.common.internals"
FrankYang0529 Apr 29, 2025
a1be1f6
KAFKA-17747: Replace Murmur3 with XXHash64
FrankYang0529 Apr 29, 2025
9b4294f
KAFKA-17747: Move static function to Utils
FrankYang0529 Apr 29, 2025
478dd98
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 Apr 30, 2025
9fc8e8e
KAFKA-17747: Replace ByteArrayOutputStream with ByteBufferOutputStream
FrankYang0529 Apr 30, 2025
167cd4a
KAFKA-17747: Add comment
FrankYang0529 Apr 30, 2025
6e219a9
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 May 1, 2025
08cdd37
KAFKA-17747: Change index to length in rack string
FrankYang0529 May 1, 2025
527b387
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 May 4, 2025
cde92d8
KAFKA-17747: remove : and , in rack string and change streams to for-…
FrankYang0529 May 4, 2025
b238375
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 May 6, 2025
a1b658a
KAFKA-17747: change to use hash4j
FrankYang0529 May 13, 2025
ed7b450
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 May 13, 2025
44c1337
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 May 13, 2025
f1ea74b
address comment
FrankYang0529 May 13, 2025
bb1edb3
address comment
FrankYang0529 May 14, 2025
3382702
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 May 14, 2025
c031a29
KAFKA-17747: handle empty map in computeGroupHash
FrankYang0529 May 14, 2025
ec7024c
Merge branch 'trunk' into KAFKA-17747-2
FrankYang0529 May 14, 2025
cf35b7a
address comment
FrankYang0529 May 14, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -1414,6 +1414,7 @@ project(':group-coordinator') {
implementation project(':coordinator-common')
implementation libs.jacksonDatabind
implementation libs.jacksonJDK8Datatypes
implementation libs.lz4
Copy link
Member

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?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed it. Thanks.

implementation libs.metrics
implementation libs.hdrHistogram
implementation libs.re2j
Expand Down
1 change: 1 addition & 0 deletions checkstyle/import-control-group-coordinator.xml
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@

<subpackage name="coordinator">
<subpackage name="group">
<allow pkg="net.jpountz.xxhash" />
<allow pkg="org.apache.kafka.clients.consumer" />
<allow pkg="org.apache.kafka.common.annotation" />
<allow pkg="org.apache.kafka.common.config" />
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,27 @@
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 net.jpountz.xxhash.XXHash64;
import net.jpountz.xxhash.XXHashFactory;

import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
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;
import java.util.function.LongFunction;
import java.util.stream.Collectors;
import java.util.stream.IntStream;

/**
* Interface common for all groups.
Expand Down Expand Up @@ -209,4 +221,90 @@ void validateOffsetFetch(
default boolean shouldExpire() {
return true;
}

/**
* The magic byte used to identify the version of topic hash function.
*/
byte TOPIC_HASH_MAGIC_BYTE = 0x00;
XXHash64 LZ4_HASH_INSTANCE = XXHashFactory.fastestInstance().hash64();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would it be possible to put those and the new methods to a separate class? Having them in Group is weird because it won't be used by all the group types.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Move to org.apache.kafka.coordinator.group.Utils. Thanks.


/**
* 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) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder whether it is worth inlining the implementation from Guava or something similar to combine the hashes. It would avoid the extra collections. I am not sure whether it makes a real difference though. What are your thoughts?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the KIP, you also mentioned combining the index with the hash. Is this something done within combineOrdered?

Copy link
Member Author

@FrankYang0529 FrankYang0529 Apr 28, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder whether it is worth inlining the implementation from Guava or something similar to combine the hashes. It would avoid the extra collections.

Yes, I copy some implementation to this function.

In the KIP, you also mentioned combining the index with the hash. Is this something done within combineOrdered?

No, the computeGroupHash sorts topics by name and use this order to merge hashes. I also add test case testComputeGroupHashWithDifferentOrder and testComputeGroupHashWithSameKeyButDifferentValue to verify it.

// 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]);
}
});

// Convert the byte array to long. This is taken from guava BytesHashCode#asLong.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not using LZ4_HASH_INSTANCE?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we can use it. Thanks.

// https://github.com/google/guava/blob/bdf2a9d05342fca852645278d474082905e09d94/guava/src/com/google/common/hash/HashCode.java#L279-L295
long retVal = (resultBytes[0] & 0xFF);
for (int i = 1; i < resultBytes.length; i++) {
retVal |= (resultBytes[i] & 0xFFL) << (i * 8);
}
return retVal;
}

/**
* Computes the hash of the topic id, name, number of partitions, and partition racks by Murmur3.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please update the docs

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated it. Thanks.

*
* @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 (ByteArrayOutputStream baos = new ByteArrayOutputStream();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can do a small optimization for it by using ByteBufferOutputStream. for example:

        try (var baos = new ByteBufferOutputStream(100);
             var dos = new DataOutputStream(baos)) {
            ...
            dos.flush();
            var topicBytes = baos.buffer().flip();
            return LZ4_HASH_INSTANCE.hash(topicBytes, 0);
        }

LZ4_HASH_INSTANCE.hash takes an array of ByteBuffer to compute the hash, which avoids an array copy.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I misunderstood ByteBufferOutputStream. I thought it uses fixed capacity even if there is no enough buffer. After checking the source code, it expands memory if the buffer is not big enough. Updated it. Thanks.

DataOutputStream dos = new DataOutputStream(baos)) {
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())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the KIP does not mention the "index" for the rack. could it be replaced by String.join(",", sortedRacksList)?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is no limitation for rack string, so any character can be part of rack string. I can update KIP if needs.

.mapToObj(idx -> idx + ":" + sortedRacksList.get(idx)) // Format: "index:value"
.collect(Collectors.joining(",")); // Separator between "index:value" pairs
dos.writeUTF(racks); // sorted racks
}
dos.flush();
byte[] topicBytes = baos.toByteArray();
return LZ4_HASH_INSTANCE.hash(topicBytes, 0, topicBytes.length, 0);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,200 @@
/*
* 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 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.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
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 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();
private static final XXHash64 LZ4_HASH_INSTANCE = XXHashFactory.fastestInstance().hash64();

@Test
void testComputeTopicHash() throws IOException {
long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster());

try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos)) {
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();
byte[] topicBytes = baos.toByteArray();
assertEquals(LZ4_HASH_INSTANCE.hash(topicBytes, 0, topicBytes.length, 0), result);
}
}

@Test
void testComputeTopicHashWithDifferentMagicByte() throws IOException {
long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster());

try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos)) {
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();
byte[] topicBytes = baos.toByteArray();
assertNotEquals(LZ4_HASH_INSTANCE.hash(topicBytes, 0, topicBytes.length, 0), result);
}
}

@Test
void testComputeTopicHashWithDifferentPartitionOrder() throws IOException {
long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster());

try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos)) {
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();
byte[] topicBytes = baos.toByteArray();
assertNotEquals(LZ4_HASH_INSTANCE.hash(topicBytes, 0, topicBytes.length, 0), result);
}
}

@Test
void testComputeTopicHashWithDifferentRackOrder() throws IOException {
long result = Group.computeTopicHash(FOO_METADATA_IMAGE.topics().getTopic(FOO_TOPIC_ID), FOO_METADATA_IMAGE.cluster());

try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos)) {
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();
byte[] topicBytes = baos.toByteArray();
assertNotEquals(LZ4_HASH_INSTANCE.hash(topicBytes, 0, topicBytes.length, 0), result);
}
}

@ParameterizedTest
@MethodSource("differentFieldGenerator")
void testComputeTopicHashWithDifferentField(MetadataImage differentImage, Uuid topicId) throws IOException {
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 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(Group.computeGroupHash(ascendTopicHashes), Group.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(Group.computeGroupHash(map1), Group.computeGroupHash(map2));
}
}