Skip to content
Merged
Show file tree
Hide file tree
Changes from 16 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 @@ -1412,6 +1412,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 @@ -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() {}
Expand Down Expand Up @@ -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) {
Copy link
Member

Choose a reason for hiding this comment

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

nit: public?

Copy link
Member Author

Choose a reason for hiding this comment

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

IIUC, this function will only be used in GroupMetadataManager like throwIfRegularExpressionIsInvalid. I think we can add public when we need it. WDYT?

Copy link
Member

Choose a reason for hiding this comment

The 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 -> {
Copy link
Member

Choose a reason for hiding this comment

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

I think we're adding a lot of unnecessary overhead for the hash computation (multiple map calls, etc.). We should probably just use an old school loop.

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

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.

Copy link
Contributor

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?

Copy link
Member

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?

yes, there is a magic byte as version.

Copy link
Member

Choose a reason for hiding this comment

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

nit: public?

Copy link
Member Author

Choose a reason for hiding this comment

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

Same as above.

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);
Copy link
Member

Choose a reason for hiding this comment

The 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?

Copy link
Member

Choose a reason for hiding this comment

The 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?

Copy link
Member

Choose a reason for hiding this comment

The 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,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();
Copy link
Member

Choose a reason for hiding this comment

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

XXH3 seems to be the fastest implementation. Did we consider using that?

https://github.com/Cyan4973/xxHash

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. 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 com.dynatrace.hash4j. Do we want to import it?

cc @chia7712 @dajac

Benchmark                                      (numReplicasPerBroker)  (partitionsPerTopic)  (replicationFactor)  Mode  Cnt      Score      Error  Units
TopicHashBenchmark.testDynatraceStreamingXXH3                      10                    10                    3  avgt    5    879.241 ±    6.788  ns/op
TopicHashBenchmark.testDynatraceStreamingXXH3                      10                    50                    3  avgt    5   4192.380 ±  195.424  ns/op
TopicHashBenchmark.testDynatraceStreamingXXH3                      10                   100                    3  avgt    5   8027.227 ±  210.403  ns/op
TopicHashBenchmark.testDynatraceXXH3                               10                    10                    3  avgt    5   1676.398 ±    2.249  ns/op
TopicHashBenchmark.testDynatraceXXH3                               10                    50                    3  avgt    5   9256.175 ±   45.298  ns/op
TopicHashBenchmark.testDynatraceXXH3                               10                   100                    3  avgt    5  20195.772 ±   37.651  ns/op
TopicHashBenchmark.testLz4StreamingXXHash64                        10                    10                    3  avgt    5   9739.833 ±  188.303  ns/op
TopicHashBenchmark.testLz4StreamingXXHash64                        10                    50                    3  avgt    5  45540.195 ±  455.747  ns/op
TopicHashBenchmark.testLz4StreamingXXHash64                        10                   100                    3  avgt    5  89084.689 ± 2164.862  ns/op
TopicHashBenchmark.testLz4XXHash64                                 10                    10                    3  avgt    5   1755.391 ±    6.436  ns/op
TopicHashBenchmark.testLz4XXHash64                                 10                    50                    3  avgt    5   9421.643 ±   79.838  ns/op
TopicHashBenchmark.testLz4XXHash64                                 10                   100                    3  avgt    5  19461.960 ±  425.881  ns/op
JMH benchmarks done
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.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
        };
    }
}

Copy link
Member

Choose a reason for hiding this comment

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

Using streaming XXH3 seems pretty interesting to me given the results. Is com.dynatrace.hash4j the only way to get it? The library seems reasonable to be taken as a dependency on the server.

Copy link
Member Author

Choose a reason for hiding this comment

The 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.

Copy link
Member

Choose a reason for hiding this comment

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

Thanks. I am fine with taking hash4j as a dependency. It is a small one without too much risk but let's see what @ijuma and @chia7712 think about it too.

Copy link
Member

Choose a reason for hiding this comment

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

It looks like a high quality library from their description of why they didn't use existing ones: https://www.dynatrace.com/news/blog/hash4j-new-library-java/

Copy link
Member

Choose a reason for hiding this comment

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

If this is a server-side dependency, it seems fine. If it's also client-side, we probably need to think more about it.

Copy link
Member

Choose a reason for hiding this comment

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

I confirm that it is sever-side only.

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 will update the PR to use hash4j with streaming XXH3. Thanks.


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

Choose a reason for hiding this comment

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

From a usage perspective, I wonder whether we should just pass the topic name and the metadata image to the method to compute the hash. I suppose that we will use it like this in the end but I am not 100% sure. 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.

If input is topic name and metadata image, we need to handle null TopicImage in computeTopicHash. Do we want to handle this error in it?

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 if we could return 0 when the topic does not exist. Would it work?

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, I think it works. I also check whether topic map is empty computeGroupHash. If it's, return 0.


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
),
Copy link
Member

Choose a reason for hiding this comment

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

nit: Could you please put new MetadataImageBuilder() // different topic id on a new line too?

            Arguments.of(
                new MetadataImageBuilder() // different topic id
                    .addTopic(differentTopicId, FOO_TOPIC_NAME, FOO_NUM_PARTITIONS)
                    .addRacks()
                    .build(),
                differentTopicId
            ),

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.

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