Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 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
8 changes: 7 additions & 1 deletion LICENSE-binary
Original file line number Diff line number Diff line change
Expand Up @@ -212,7 +212,10 @@ License Version 2.0:
- commons-lang3-3.12.0
- commons-logging-1.3.2
- commons-validator-1.9.0
- error_prone_annotations-2.21.1
- error_prone_annotations-2.36.0
- failureaccess-1.0.2
- guava-33.4.0-jre
- j2objc-annotations-3.0.0
- jackson-annotations-2.16.2
- jackson-core-2.16.2
- jackson-databind-2.16.2
Expand All @@ -237,6 +240,7 @@ License Version 2.0:
- jetty-session-12.0.15
- jetty-util-12.0.15
- jose4j-0.9.4
- listenablefuture-9999.0-empty-to-avoid-conflict-with-guava
- log4j-api-2.24.3
- log4j-core-2.24.3
- log4j-slf4j-impl-2.24.3
Expand Down Expand Up @@ -298,6 +302,7 @@ see: licenses/CDDL+GPL-1.1
MIT License

- argparse4j-0.7.0, see: licenses/argparse-MIT
- checker-qual-3.43.0, see: licenses/checker-qual-MIT
- classgraph-4.8.173, see: licenses/classgraph-MIT
- jopt-simple-5.0.4, see: licenses/jopt-simple-MIT
- slf4j-api-1.7.36, see: licenses/slf4j-MIT
Expand All @@ -313,6 +318,7 @@ BSD 2-Clause
BSD 3-Clause

- jline-3.25.1, see: licenses/jline-BSD-3-clause
- jsr305-3.0.2, see: licenses/jsr305-BSD-3-clause
- protobuf-java-3.25.5, see: licenses/protobuf-java-BSD-3-clause
- jakarta.activation-2.0.1, see: licenses/jakarta-BSD-3-clause

Expand Down
1 change: 1 addition & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -1420,6 +1420,7 @@ project(':group-coordinator') {
implementation libs.hdrHistogram
implementation libs.re2j
implementation libs.slf4jApi
implementation libs.guava

testImplementation project(':clients').sourceSets.test.output
testImplementation project(':server-common').sourceSets.test.output
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 @@ -77,6 +77,7 @@
<allow pkg="org.apache.kafka.coordinator.common.runtime" />
<allow pkg="com.google.re2j" />
<allow pkg="org.apache.kafka.metadata" />
<allow pkg="com.google.common.hash" />
<subpackage name="metrics">
<allow pkg="com.yammer.metrics"/>
<allow pkg="org.HdrHistogram" />
Expand Down
2 changes: 1 addition & 1 deletion committer-tools/verify_license.py
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
# DependencyName-x.y, DependencyName-x.y.z, or DependencyName-x.y.z.w
# Optionally, a trailing suffix (e.g., "-alpha") is captured.
LICENSE_DEP_PATTERN = re.compile(
r'^\s*-\s*([A-Za-z0-9_.+-]+-[0-9]+\.[0-9]+(?:\.[0-9]+){0,2}(?:[-.][A-Za-z0-9]+)?)',
r'^\s*-\s*([A-Za-z0-9_.+-]+-[0-9]+\.[0-9]+(?:\.[0-9]+){0,2}(?:[-.][A-Za-z0-9]+)*)',
re.MULTILINE
)

Expand Down
2 changes: 2 additions & 0 deletions gradle/dependencies.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ versions += [
classgraph: "4.8.173",
gradle: "8.10.2",
grgit: "4.1.1",
guava: "33.4.0-jre",
httpclient: "4.5.14",
jackson: "2.16.2",
jacoco: "0.8.10",
Expand Down Expand Up @@ -147,6 +148,7 @@ libs += [
caffeine: "com.github.ben-manes.caffeine:caffeine:$versions.caffeine",
classgraph: "io.github.classgraph:classgraph:$versions.classgraph",
commonsValidator: "commons-validator:commons-validator:$versions.commonsValidator",
guava: "com.google.guava:guava:$versions.guava",
Copy link
Member

Choose a reason for hiding this comment

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

This is something that we haven't really discussed in the KIP because it is an implementation detail but we should discuss whether we really want to take a dependency on Guava.

Copy link
Member Author

Choose a reason for hiding this comment

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

I update PR to remove guava. I think we can put all data to a byte array and use Murmur3 to hash it, so we don't rely on guava.

Copy link
Member

Choose a reason for hiding this comment

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

Thanks!

While talking to @ijuma about it, he has suggested to look into https://github.com/lz4/lz4-java/tree/master/src/java/net/jpountz/xxhash. We get it via lz4 and it is apparently much faster than Murmur3. It may be worth running a few benchmarks to compare then. What do you think?

I also wonder what is the impact of putting all the data to a byte array before hashing it. Do you have thoughts on this?

Copy link
Member

Choose a reason for hiding this comment

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

IIRC, the lz4-java is not maintained anymore, so not sure whether the code maintaining is a risk.

I also wonder what is the impact of putting all the data to a byte array before hashing it. Do you have thoughts on this?

I suggest that EventProcessorThread can leverage GrowableBufferSupplier to reuse buffer as much as possible. Additionally, Group#computeTopicHashin should use ByteBufferOutputStream to generate the bytes array, as ByteBufferOutputStream#buffer#array can avoid extra array copy like ByteArrayOutputStream#toByteArray

Copy link
Member

Choose a reason for hiding this comment

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

This hash function is used by zstd too. Its pretty safe to rely on it given that lz4 and zstd are the most popular compression algorithms. And we will be supporting them for the foreseeable future.

Which particular implementation we use is a fair question.

Copy link
Member

Choose a reason for hiding this comment

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

IIRC, the lz4-java is not maintained anymore, so not sure whether the code maintaining is a risk.

I also wonder what is the impact of putting all the data to a byte array before hashing it. Do you have thoughts on this?

I suggest that EventProcessorThread can leverage GrowableBufferSupplier to reuse buffer as much as possible. Additionally, Group#computeTopicHashin should use ByteBufferOutputStream to generate the bytes array, as ByteBufferOutputStream#buffer#array can avoid extra array copy like ByteArrayOutputStream#toByteArray

I agree with using a BufferSupplier in order to reuse buffers. However, EventProcessorThread may be too low level to hold it. Having it in Shard may be enough.

Copy link
Member Author

Choose a reason for hiding this comment

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

Thanks for the suggestion. Updated benchmark result.

Benchmark                      (numReplicasPerBroker)  (partitionsPerTopic)  (replicationFactor)  Mode  Cnt     Score    Error  Units
TopicHashBenchmark.testLz4                         10                    10                    3  avgt   15   166.389 ±  1.542  ns/op
TopicHashBenchmark.testLz4                         10                    50                    3  avgt   15   375.660 ±  2.771  ns/op
TopicHashBenchmark.testLz4                         10                   100                    3  avgt   15   636.176 ±  8.305  ns/op
TopicHashBenchmark.testMurmur                      10                    10                    3  avgt   15   238.242 ±  1.664  ns/op
TopicHashBenchmark.testMurmur                      10                    50                    3  avgt   15  1143.583 ±  5.981  ns/op
TopicHashBenchmark.testMurmur                      10                   100                    3  avgt   15  2278.680 ± 29.007  ns/op
TopicHashBenchmark.java
package org.apache.kafka.jmh.metadata;

import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.RegisterBrokerRecord;
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 org.apache.kafka.streams.state.internals.Murmur3;

import net.jpountz.xxhash.XXHash64;
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.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;

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 = 5)
@Measurement(iterations = 15)
@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 byte[] topicBytes;

    @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 topicImage = topicsDelta.apply().topicsById().values().stream().findFirst().get();
        ClusterImage clusterImage = clusterDelta.apply();

        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
             DataOutputStream dos = new DataOutputStream(baos)) {
            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
                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())
                    .mapToObj(idx -> idx + ":" + sortedRacksList.get(idx)) // Format: "index:value"
                    .collect(Collectors.joining(",")); // Separator between "index:value" pairs
                dos.writeUTF(racks); // sorted racks
            }
            dos.flush();
            topicBytes = baos.toByteArray();
        }
    }

    @Benchmark
    public void testLz4() {
        XXHash64 hash = XXHashFactory.fastestInstance().hash64();
        hash.hash(topicBytes, 0, topicBytes.length, 0);
    }

    @Benchmark
    public void testMurmur() {
        Murmur3.hash64(topicBytes);
    }
}

Copy link
Member

Choose a reason for hiding this comment

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

I agree with using a BufferSupplier in order to reuse buffers. However, EventProcessorThread may be too low level to hold it. Having it in Shard may be enough.

we can revisit this when the critical code are used by production :)

@FrankYang0529 thanks for updates. the result LGTM.

Copy link
Member Author

Choose a reason for hiding this comment

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

I also wonder what is the impact of putting all the data to a byte array before hashing it. Do you have thoughts on this?

Based on the KIP-1101, it minimizes the calculation count of topic hash. The result can be shared between groups. I think we can keep this function simple currently.

I suggest that EventProcessorThread can leverage GrowableBufferSupplier to reuse buffer as much as possible.

With BufferSupplier, the hash function needs to be thread safe to reuse the buffer. We can revisit it in the future.

Additionally, Group#computeTopicHashin should use ByteBufferOutputStream to generate the bytes array, as ByteBufferOutputStream#buffer#array can avoid extra array copy like ByteArrayOutputStream#toByteArray

The ByteBufferOutputStream needs a ByteBuffer with capacity. I wonder whether we can calculate a accurate capacity. For example, rack string can contain any character. It presents 1 to 4 bytes in UTF-8.

Copy link
Member

Choose a reason for hiding this comment

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

The ByteBufferOutputStream needs a ByteBuffer with capacity. I wonder whether we can calculate a accurate capacity. For example, rack string can contain any character. It presents 1 to 4 bytes in UTF-8.

The initialize capacity can be discussed later. In fact, it may be not a issue if we adopt the growable buffer. The buffer can be big enough for each hash computing eventually.

jacksonAnnotations: "com.fasterxml.jackson.core:jackson-annotations:$versions.jackson",
jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson",
jacksonDatabindYaml: "com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:$versions.jackson",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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) {
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.

return Hashing.combineOrdered(
topicHashes.entrySet()
.stream()
.sorted(Map.Entry.comparingByKey())
.map(e -> HashCode.fromLong(e.getValue()))
.toList()
).asLong();
}

/**
* 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) {
HashFunction hf = Hashing.murmur3_128();
Hasher topicHasher = hf.newHasher()
.putByte((byte) 0) // magic byte
Copy link
Member

Choose a reason for hiding this comment

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

Should we define a constant for the magic byte?

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, add TOPIC_HASH_MAGIC_BYTE.

.putLong(topicImage.id().hashCode()) // topic Id
.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 -> {
Copy link
Member

Choose a reason for hiding this comment

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

We know that partitions go from 0 to N. I wonder whether we should use a good old for loop instead of sorting the partitions. What do you think?

Copy link
Member Author

Choose a reason for hiding this comment

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

Good suggestion! Thanks. Updated it.

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)
.sorted()
.collect(Collectors.joining(";"));
Copy link
Member

Choose a reason for hiding this comment

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

; is allowed in the rack field too so it does really protect us.

Copy link
Member Author

Choose a reason for hiding this comment

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

It looks like any character can be valid. I change the combination with following format:

0:<rack 0>,1:<rack 1>, ...

Copy link
Contributor

@squah-confluent squah-confluent Apr 30, 2025

Choose a reason for hiding this comment

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

I think this is fine preventing for accidental collisions. Though it's still possible to intentionally come up with rack names that create collisions, but I believe you'd only be impacting your own cluster.

To rule out any ambiguity, we could pretend this was a serialization format and either prefix strings with their length, or null-terminate them. The same for variable-length lists of strings. These can either be length-prefixed or terminated with an invalid string that cannot occur (""? but not sure on this).

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for taking the suggestion. I think it's fine now.
Small nit though, I was actually thinking of writing the length in binary, using writeInt and dropping the : and , separators entirely. Apologies if I wasn't clear enough earlier.

Copy link
Member

Choose a reason for hiding this comment

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

Looking at the source code, putString already handles he collision issue by appending the length:

  @Override
  default HashStream putString(String s) {
    putChars(s);
    putInt(s.length());
    return this;
  }

https://github.com/dynatrace-oss/hash4j/blob/main/src/main/java/com/dynatrace/hash4j/hashing/AbstractHashStream.java

The cost of this extra length info is negligible, so we don't need to change the logic. However, it would be helpful to amend the comments to clarify that.

WDYT?

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
.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);
}
}
22 changes: 22 additions & 0 deletions licenses/checker-qual-MIT
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
Checker Framework qualifiers
Copyright 2004-present by the Checker Framework developers

MIT License:

Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:

The above copyright notice and this permission notice shall be included in
all copies or substantial portions of the Software.

THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.