This is an automated email from the ASF dual-hosted git repository.

chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new b347f4bd2e6 KAFKA-20297 move ImplicitLinkedHashCollection, 
ImplicitLinkedHashMultiCollection, and ByteUtils from utils to internals 
(#21856)
b347f4bd2e6 is described below

commit b347f4bd2e6aceb424dad6fe24fd9cc6276befc5
Author: Chia-Yi Chiu <[email protected]>
AuthorDate: Tue Mar 31 00:57:48 2026 +0900

    KAFKA-20297 move ImplicitLinkedHashCollection, 
ImplicitLinkedHashMultiCollection, and ByteUtils from utils to internals 
(#21856)
    
    Summary: Move ImplicitLinkedHash*, ByteUtils from common.utils to
    common.utils.internals.
    
    Reviewers: Chia-Ping Tsai <[email protected]>
---
 .../common/compress/Lz4BlockOutputStream.java      |  2 +-
 .../kafka/common/network/SslTransportLayer.java    |  2 +-
 .../kafka/common/protocol/ByteBufferAccessor.java  |  2 +-
 .../common/protocol/DataOutputStreamWritable.java  |  2 +-
 .../apache/kafka/common/protocol/SendBuilder.java  |  2 +-
 .../common/protocol/types/CompactArrayOf.java      |  2 +-
 .../kafka/common/protocol/types/TaggedFields.java  |  2 +-
 .../apache/kafka/common/protocol/types/Type.java   |  2 +-
 .../record/internal/AbstractLegacyRecordBatch.java |  2 +-
 .../common/record/internal/DefaultRecord.java      |  2 +-
 .../common/record/internal/DefaultRecordBatch.java |  2 +-
 .../kafka/common/record/internal/LegacyRecord.java |  2 +-
 .../java/org/apache/kafka/common/utils/Bytes.java  | 14 ++--
 .../common/utils/{ => internals}/ByteUtils.java    | 66 ++++++++++++++-
 .../kafka/common/utils/internals/BytesUtils.java   | 93 ----------------------
 .../ImplicitLinkedHashCollection.java              |  2 +-
 .../ImplicitLinkedHashMultiCollection.java         |  4 +-
 .../common/message/SimpleExampleMessageTest.java   |  2 +-
 .../protocol/types/ProtocolSerializationTest.java  |  2 +-
 .../common/record/internal/DefaultRecordTest.java  |  2 +-
 .../record/internal/EndTransactionMarkerTest.java  |  2 +-
 .../org/apache/kafka/common/utils/BytesTest.java   | 10 +--
 .../utils/{ => internals}/ByteUtilsTest.java       |  6 +-
 .../ImplicitLinkedHashCollectionTest.java          | 10 +--
 .../ImplicitLinkedHashMultiCollectionTest.java     | 12 +--
 .../kafka/server/share/SharePartitionManager.java  |  2 +-
 .../server/share/SharePartitionManagerTest.java    |  2 +-
 .../unit/kafka/server/EdgeCaseRequestTest.scala    |  2 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala    |  3 +-
 .../org/apache/kafka/message/MessageGenerator.java |  6 +-
 gradle/spotbugs-exclude.xml                        |  2 +-
 .../ImplicitLinkedHashCollectionBenchmark.java     |  2 +-
 .../jmh/streams/RawBytesExtractionBenchmark.java   |  2 +-
 .../apache/kafka/jmh/util/ByteUtilsBenchmark.java  |  2 +-
 .../kafka/jmh/util/BytesCompareBenchmark.java      |  6 +-
 .../kafka/metadata/MetadataRecordSerdeTest.java    |  2 +-
 .../org/apache/kafka/metadata/RecordTestUtils.java |  2 +-
 .../apache/kafka/raft/internals/BatchBuilder.java  |  2 +-
 .../kafka/raft/internals/RecordsIterator.java      |  2 +-
 .../kafka/raft/internals/BatchAccumulatorTest.java |  2 +-
 .../serialization/AbstractApiMessageSerde.java     |  2 +-
 .../java/org/apache/kafka/server/FetchContext.java |  2 +-
 .../java/org/apache/kafka/server/FetchSession.java |  2 +-
 .../kafka/server/FetchSessionCacheShard.java       |  2 +-
 .../kafka/server/share/CachedSharePartition.java   |  2 +-
 .../kafka/server/share/session/ShareSession.java   |  2 +-
 .../server/share/session/ShareSessionCache.java    |  2 +-
 .../org/apache/kafka/server/FetchSessionTest.java  |  2 +-
 .../share/session/ShareSessionCacheTest.java       |  2 +-
 .../internals/log/ProducerStateManager.java        |  2 +-
 .../storage/internals/log/SkimpyOffsetMap.java     |  2 +-
 .../kstream/internals/ChangedDeserializer.java     |  2 +-
 .../kstream/internals/ChangedSerializer.java       |  2 +-
 .../ForeignTableJoinProcessorSupplier.java         |  4 +-
 .../AggregationWithHeadersSerializer.java          |  2 +-
 .../state/internals/CachingKeyValueStore.java      |  4 +-
 .../state/internals/DualColumnFamilyAccessor.java  |  6 +-
 .../state/internals/HeadersDeserializer.java       |  2 +-
 .../streams/state/internals/HeadersSerializer.java |  2 +-
 .../state/internals/InMemoryKeyValueStore.java     |  4 +-
 .../state/internals/MemoryNavigableLRUCache.java   |  4 +-
 .../streams/state/internals/RecordConverters.java  |  2 +-
 .../state/internals/RocksDBRangeIterator.java      |  4 +-
 .../streams/state/internals/RocksDBStore.java      |  8 +-
 .../state/internals/SegmentedCacheFunction.java    |  4 +-
 .../kafka/streams/state/internals/Utils.java       |  2 +-
 .../ValueTimestampHeadersDeserializer.java         |  2 +-
 .../internals/ValueTimestampHeadersSerializer.java |  2 +-
 .../kstream/internals/ChangedSerdeTest.java        |  2 +-
 .../kafka/streams/state/HeadersBytesStoreTest.java |  2 +-
 .../ChangeLoggingKeyValueBytesStoreTest.java       |  4 +-
 .../state/internals/RocksDBRangeIteratorTest.java  |  6 +-
 .../TimeOrderedSessionStoreUpgradeTest.java        |  2 +-
 .../TimeOrderedWindowStoreUpgradeTest.java         |  2 +-
 .../kafka/streams/state/internals/UtilsTest.java   |  2 +-
 75 files changed, 181 insertions(+), 205 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/compress/Lz4BlockOutputStream.java
 
b/clients/src/main/java/org/apache/kafka/common/compress/Lz4BlockOutputStream.java
index 8d96afd2737..8137fcb9279 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/compress/Lz4BlockOutputStream.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/compress/Lz4BlockOutputStream.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.common.compress;
 
 import org.apache.kafka.common.record.internal.CompressionType;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import net.jpountz.lz4.LZ4Compressor;
 import net.jpountz.lz4.LZ4Factory;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java 
b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
index d047c79ce8a..a52f19469a1 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java
@@ -19,9 +19,9 @@ package org.apache.kafka.common.network;
 import org.apache.kafka.common.errors.SslAuthenticationException;
 import org.apache.kafka.common.security.auth.KafkaPrincipal;
 import org.apache.kafka.common.utils.ByteBufferUnmapper;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.slf4j.Logger;
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java
 
b/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java
index c3e2886e656..160cf272c78 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java
@@ -17,7 +17,7 @@
 
 package org.apache.kafka.common.protocol;
 
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 
diff --git 
a/clients/src/main/java/org/apache/kafka/common/protocol/DataOutputStreamWritable.java
 
b/clients/src/main/java/org/apache/kafka/common/protocol/DataOutputStreamWritable.java
index dcf53c4e52b..e47e966e1a7 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/protocol/DataOutputStreamWritable.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/protocol/DataOutputStreamWritable.java
@@ -17,8 +17,8 @@
 
 package org.apache.kafka.common.protocol;
 
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.io.Closeable;
 import java.io.DataOutputStream;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/protocol/SendBuilder.java 
b/clients/src/main/java/org/apache/kafka/common/protocol/SendBuilder.java
index 11239d8e38e..509243b2237 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/SendBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/SendBuilder.java
@@ -24,7 +24,7 @@ import 
org.apache.kafka.common.record.internal.MultiRecordsSend;
 import org.apache.kafka.common.record.internal.UnalignedMemoryRecords;
 import org.apache.kafka.common.requests.RequestHeader;
 import org.apache.kafka.common.requests.ResponseHeader;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/protocol/types/CompactArrayOf.java
 
b/clients/src/main/java/org/apache/kafka/common/protocol/types/CompactArrayOf.java
index e41a4c0dc7f..219858c28ed 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/protocol/types/CompactArrayOf.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/protocol/types/CompactArrayOf.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.common.protocol.types;
 
 import org.apache.kafka.common.protocol.types.Type.DocumentedType;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 import java.util.Optional;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/protocol/types/TaggedFields.java
 
b/clients/src/main/java/org/apache/kafka/common/protocol/types/TaggedFields.java
index 521f6346ed9..4ffa3b4bcb5 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/protocol/types/TaggedFields.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/protocol/types/TaggedFields.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.common.protocol.types;
 
 import org.apache.kafka.common.protocol.types.Type.DocumentedType;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 import java.util.Collections;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java 
b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
index 2a9c594cfe4..b2919dc52f2 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
@@ -19,8 +19,8 @@ package org.apache.kafka.common.protocol.types;
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.record.internal.BaseRecords;
 import org.apache.kafka.common.record.internal.MemoryRecords;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 import java.util.Optional;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/AbstractLegacyRecordBatch.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/AbstractLegacyRecordBatch.java
index aca803cf5f3..7738f4059d8 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/AbstractLegacyRecordBatch.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/AbstractLegacyRecordBatch.java
@@ -25,9 +25,9 @@ import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.AbstractIterator;
 import org.apache.kafka.common.utils.BufferSupplier;
 import org.apache.kafka.common.utils.ByteBufferOutputStream;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.CloseableIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/DefaultRecord.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/DefaultRecord.java
index 8a3e37e839a..e8241eb0b70 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/DefaultRecord.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/DefaultRecord.java
@@ -20,8 +20,8 @@ import org.apache.kafka.common.InvalidRecordException;
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.internals.RecordHeader;
 import org.apache.kafka.common.record.TimestampType;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/DefaultRecordBatch.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/DefaultRecordBatch.java
index acc590fd953..2b6bc7ee11f 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/DefaultRecordBatch.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/DefaultRecordBatch.java
@@ -24,9 +24,9 @@ import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.BufferSupplier;
 import org.apache.kafka.common.utils.ByteBufferOutputStream;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.CloseableIterator;
 import org.apache.kafka.common.utils.Crc32C;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.io.IOException;
 import java.io.InputStream;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/LegacyRecord.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/LegacyRecord.java
index a481380c15b..1f5d7d825ff 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/LegacyRecord.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/LegacyRecord.java
@@ -20,9 +20,9 @@ import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.errors.CorruptRecordException;
 import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.ByteBufferOutputStream;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Checksums;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java 
b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java
index 5791deec3d9..24cfe3c8079 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.common.utils;
 
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.io.Serializable;
 import java.util.Arrays;
@@ -113,7 +113,7 @@ public class Bytes implements Comparable<Bytes> {
 
     @Override
     public int compareTo(Bytes that) {
-        return BytesUtils.BYTES_LEXICO_COMPARATOR.compare(this.bytes, 
that.bytes);
+        return ByteUtils.BYTES_LEXICO_COMPARATOR.compare(this.bytes, 
that.bytes);
     }
 
     @Override
@@ -166,17 +166,17 @@ public class Bytes implements Comparable<Bytes> {
      * @return A new copy of the incremented byte array.
      * @throws IndexOutOfBoundsException if incrementing causes the underlying 
input byte array to overflow.
      * @deprecated This method is not part of the public API and will be 
removed in version 5.0.
-     *             Internal Kafka code should use {@link 
org.apache.kafka.common.utils.internals.BytesUtils#increment(Bytes)} instead.
+     *             Internal Kafka code should use {@link 
org.apache.kafka.common.utils.internals.ByteUtils#increment(Bytes)} instead.
      */
     @Deprecated(since = "4.3", forRemoval = true)
     public static Bytes increment(Bytes input) throws 
IndexOutOfBoundsException {
-        return BytesUtils.increment(input);
+        return ByteUtils.increment(input);
     }
 
     /**
      * A byte array comparator based on lexicographic ordering.
      * @deprecated This field is not part of the public API and will be 
removed in version 5.0.
-     *             Internal Kafka code should use {@link 
org.apache.kafka.common.utils.internals.BytesUtils#BYTES_LEXICO_COMPARATOR} 
instead.
+     *             Internal Kafka code should use {@link 
org.apache.kafka.common.utils.internals.ByteUtils#BYTES_LEXICO_COMPARATOR} 
instead.
      */
     @Deprecated(since = "4.3", forRemoval = true)
     public static final ByteArrayComparator BYTES_LEXICO_COMPARATOR = new 
LexicographicByteArrayComparator();
@@ -185,7 +185,7 @@ public class Bytes implements Comparable<Bytes> {
      * A byte array comparator interface.
      *
      * @deprecated This interface is not part of the public API and will be 
removed in version 5.0.
-     *             Internal Kafka code should use {@link 
org.apache.kafka.common.utils.internals.BytesUtils.ByteArrayComparator} instead.
+     *             Internal Kafka code should use {@link 
org.apache.kafka.common.utils.internals.ByteUtils.ByteArrayComparator} instead.
      */
     @Deprecated(since = "4.3", forRemoval = true)
     public interface ByteArrayComparator extends Comparator<byte[]>, 
Serializable {
@@ -194,6 +194,6 @@ public class Bytes implements Comparable<Bytes> {
                     final byte[] buffer2, int offset2, int length2);
     }
 
-    private static class LexicographicByteArrayComparator extends 
BytesUtils.LexicographicByteArrayComparator implements ByteArrayComparator {
+    private static class LexicographicByteArrayComparator extends 
ByteUtils.LexicographicByteArrayComparator implements ByteArrayComparator {
     }
 }
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/ByteUtils.java
similarity index 90%
rename from clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/ByteUtils.java
index 4554d83f015..9a20b8cb91f 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/ByteUtils.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/ByteUtils.java
@@ -14,7 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.kafka.common.utils;
+package org.apache.kafka.common.utils.internals;
+
+import org.apache.kafka.common.utils.Bytes;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -23,6 +25,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
 
 /**
  * This classes exposes low-level methods for reading/writing from byte 
streams or buffers.
@@ -36,6 +40,66 @@ public final class ByteUtils {
 
     private ByteUtils() {}
 
+    /**
+     * Increment the underlying byte array by adding 1.
+     *
+     * @param input - The byte array to increment
+     * @return A new copy of the incremented byte array
+     * @throws IndexOutOfBoundsException if incrementing causes the underlying 
input byte array to overflow
+     */
+    public static Bytes increment(Bytes input) throws 
IndexOutOfBoundsException {
+        byte[] inputArr = input.get();
+        byte[] ret = new byte[inputArr.length];
+        int carry = 1;
+        for (int i = inputArr.length - 1; i >= 0; i--) {
+            if (inputArr[i] == (byte) 0xFF && carry == 1) {
+                ret[i] = (byte) 0x00;
+            } else {
+                ret[i] = (byte) (inputArr[i] + carry);
+                carry = 0;
+            }
+        }
+        if (carry == 0) {
+            return Bytes.wrap(ret);
+        } else {
+            throw new IndexOutOfBoundsException();
+        }
+    }
+
+    /**
+     * A byte array comparator based on lexicographic ordering.
+     */
+    public static final ByteArrayComparator BYTES_LEXICO_COMPARATOR = new 
LexicographicByteArrayComparator();
+
+    public interface ByteArrayComparator extends Comparator<byte[]> {
+
+        int compare(final byte[] buffer1, int offset1, int length1,
+                    final byte[] buffer2, int offset2, int length2);
+    }
+
+    public static class LexicographicByteArrayComparator implements 
ByteArrayComparator {
+
+        @Override
+        public int compare(byte[] buffer1, byte[] buffer2) {
+            return compare(buffer1, 0, buffer1.length, buffer2, 0, 
buffer2.length);
+        }
+
+        public int compare(final byte[] buffer1, int offset1, int length1,
+                           final byte[] buffer2, int offset2, int length2) {
+
+            // short circuit equal case
+            if (buffer1 == buffer2 &&
+                    offset1 == offset2 &&
+                    length1 == length2) {
+                return 0;
+            }
+
+            int end1 = offset1 + length1;
+            int end2 = offset2 + length2;
+            return Arrays.compareUnsigned(buffer1, offset1, end1, buffer2, 
offset2, end2);
+        }
+    }
+
     /**
      * Read an unsigned integer from the current position in the buffer, 
incrementing the position by 4 bytes
      *
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/internals/BytesUtils.java 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/BytesUtils.java
deleted file mode 100644
index 4823905aa4b..00000000000
--- 
a/clients/src/main/java/org/apache/kafka/common/utils/internals/BytesUtils.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * 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.common.utils.internals;
-
-import org.apache.kafka.common.utils.Bytes;
-
-import java.util.Arrays;
-import java.util.Comparator;
-
-/**
- * Internal utility class for Bytes-related operations.
- * This class is for internal Kafka use only and is not part of the public API.
- */
-public final class BytesUtils {
-
-    private BytesUtils() {
-        // Utility class, prevent instantiation
-    }
-
-    /**
-     * Increment the underlying byte array by adding 1.
-     *
-     * @param input - The byte array to increment
-     * @return A new copy of the incremented byte array
-     * @throws IndexOutOfBoundsException if incrementing causes the underlying 
input byte array to overflow
-     */
-    public static Bytes increment(Bytes input) throws 
IndexOutOfBoundsException {
-        byte[] inputArr = input.get();
-        byte[] ret = new byte[inputArr.length];
-        int carry = 1;
-        for (int i = inputArr.length - 1; i >= 0; i--) {
-            if (inputArr[i] == (byte) 0xFF && carry == 1) {
-                ret[i] = (byte) 0x00;
-            } else {
-                ret[i] = (byte) (inputArr[i] + carry);
-                carry = 0;
-            }
-        }
-        if (carry == 0) {
-            return Bytes.wrap(ret);
-        } else {
-            throw new IndexOutOfBoundsException();
-        }
-    }
-
-    /**
-     * A byte array comparator based on lexicographic ordering.
-     */
-    public static final ByteArrayComparator BYTES_LEXICO_COMPARATOR = new 
LexicographicByteArrayComparator();
-
-    public interface ByteArrayComparator extends Comparator<byte[]> {
-
-        int compare(final byte[] buffer1, int offset1, int length1,
-                    final byte[] buffer2, int offset2, int length2);
-    }
-
-    public static class LexicographicByteArrayComparator implements 
ByteArrayComparator {
-
-        @Override
-        public int compare(byte[] buffer1, byte[] buffer2) {
-            return compare(buffer1, 0, buffer1.length, buffer2, 0, 
buffer2.length);
-        }
-
-        public int compare(final byte[] buffer1, int offset1, int length1,
-                           final byte[] buffer2, int offset2, int length2) {
-
-            // short circuit equal case
-            if (buffer1 == buffer2 &&
-                    offset1 == offset2 &&
-                    length1 == length2) {
-                return 0;
-            }
-
-            int end1 = offset1 + length1;
-            int end2 = offset2 + length2;
-            return Arrays.compareUnsigned(buffer1, offset1, end1, buffer2, 
offset2, end2);
-        }
-    }
-}
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollection.java
 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashCollection.java
similarity index 99%
rename from 
clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollection.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashCollection.java
index 524b20b7188..284e1055b41 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollection.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashCollection.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.kafka.common.utils;
+package org.apache.kafka.common.utils.internals;
 
 import java.util.AbstractCollection;
 import java.util.AbstractSequentialList;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollection.java
 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashMultiCollection.java
similarity index 97%
rename from 
clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollection.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashMultiCollection.java
index 6e0d28adb4c..17252a5285c 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollection.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashMultiCollection.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.kafka.common.utils;
+package org.apache.kafka.common.utils.internals;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -24,7 +24,7 @@ import java.util.List;
 
 /**
  * A memory-efficient hash multiset which tracks the order of insertion of 
elements.
- * See org.apache.kafka.common.utils.ImplicitLinkedHashCollection for 
implementation details.
+ * See org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection 
for implementation details.
  *
  * This class is a multi-set because it allows multiple elements to be 
inserted that
  * have equivalent keys.
diff --git 
a/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java
 
b/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java
index ba3bc23bf60..2277891adc1 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/message/SimpleExampleMessageTest.java
@@ -21,7 +21,7 @@ import 
org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.protocol.ByteBufferAccessor;
 import org.apache.kafka.common.protocol.MessageUtil;
 import org.apache.kafka.common.protocol.ObjectSerializationCache;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import com.fasterxml.jackson.databind.JsonNode;
 
diff --git 
a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
 
b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
index de4e58f5d96..14bd2ae2982 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.common.protocol.types;
 
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
diff --git 
a/clients/src/test/java/org/apache/kafka/common/record/internal/DefaultRecordTest.java
 
b/clients/src/test/java/org/apache/kafka/common/record/internal/DefaultRecordTest.java
index 5e73c1df435..4afe2767b04 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/record/internal/DefaultRecordTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/record/internal/DefaultRecordTest.java
@@ -21,7 +21,7 @@ import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.internals.RecordHeader;
 import org.apache.kafka.common.utils.ByteBufferInputStream;
 import org.apache.kafka.common.utils.ByteBufferOutputStream;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.junit.jupiter.api.Test;
 
diff --git 
a/clients/src/test/java/org/apache/kafka/common/record/internal/EndTransactionMarkerTest.java
 
b/clients/src/test/java/org/apache/kafka/common/record/internal/EndTransactionMarkerTest.java
index 7438e9bba2c..df6478af792 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/record/internal/EndTransactionMarkerTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/record/internal/EndTransactionMarkerTest.java
@@ -22,7 +22,7 @@ import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.protocol.types.Type;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.junit.jupiter.api.Test;
 
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/BytesTest.java 
b/clients/src/test/java/org/apache/kafka/common/utils/BytesTest.java
index c68527243cd..d3aee800ed2 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/BytesTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/BytesTest.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.common.utils;
 
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.junit.jupiter.api.Test;
 
@@ -36,14 +36,14 @@ public class BytesTest {
     public void testIncrement() {
         byte[] input = new byte[]{(byte) 0xAB, (byte) 0xCD, (byte) 0xFF};
         byte[] expected = new byte[]{(byte) 0xAB, (byte) 0xCE, (byte) 0x00};
-        Bytes output = BytesUtils.increment(Bytes.wrap(input));
+        Bytes output = ByteUtils.increment(Bytes.wrap(input));
         assertArrayEquals(output.get(), expected);
     }
 
     @Test
     public void testIncrementUpperBoundary() {
         byte[] input = new byte[]{(byte) 0xFF, (byte) 0xFF, (byte) 0xFF};
-        assertThrows(IndexOutOfBoundsException.class, () -> 
BytesUtils.increment(Bytes.wrap(input)));
+        assertThrows(IndexOutOfBoundsException.class, () -> 
ByteUtils.increment(Bytes.wrap(input)));
     }
 
     @Test
@@ -66,7 +66,7 @@ public class BytesTest {
         map.put(key5, val);
 
         Bytes prefix = key1;
-        Bytes prefixEnd = BytesUtils.increment(prefix);
+        Bytes prefixEnd = ByteUtils.increment(prefix);
 
         Comparator<? super Bytes> comparator = map.comparator();
         final int result = comparator == null ? prefix.compareTo(prefixEnd) : 
comparator.compare(prefix, prefixEnd);
@@ -114,7 +114,7 @@ public class BytesTest {
     }
 
     private int cmp(String l, String r) {
-        return BytesUtils.BYTES_LEXICO_COMPARATOR.compare(
+        return ByteUtils.BYTES_LEXICO_COMPARATOR.compare(
             l.getBytes(StandardCharsets.UTF_8),
             r.getBytes(StandardCharsets.UTF_8));
     }
diff --git 
a/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/ByteUtilsTest.java
similarity index 99%
rename from 
clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java
rename to 
clients/src/test/java/org/apache/kafka/common/utils/internals/ByteUtilsTest.java
index 1388caba568..e8d92362d47 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/ByteUtilsTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/ByteUtilsTest.java
@@ -14,7 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.kafka.common.utils;
+package org.apache.kafka.common.utils.internals;
+
+import org.apache.kafka.common.utils.ByteBufferInputStream;
+import org.apache.kafka.common.utils.ByteBufferOutputStream;
+import org.apache.kafka.common.utils.Utils;
 
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
diff --git 
a/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollectionTest.java
 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashCollectionTest.java
similarity index 98%
rename from 
clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollectionTest.java
rename to 
clients/src/test/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashCollectionTest.java
index 4491fc88fc1..fc43fa94878 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashCollectionTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashCollectionTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.kafka.common.utils;
+package org.apache.kafka.common.utils.internals;
 
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
@@ -138,12 +138,12 @@ public class ImplicitLinkedHashCollectionTest {
         int i = 0;
         while (iterator.hasNext()) {
             TestElement element = iterator.next();
-            assertTrue(i < sequence.length, "Iterator yieled " + (i + 1) + " 
elements, but only " +
+            assertTrue(i < sequence.length, "Iterator yielded " + (i + 1) + " 
elements, but only " +
                 sequence.length + " were expected.");
             assertEquals(sequence[i].intValue(), element.key, "Iterator value 
number " + (i + 1) + " was incorrect.");
             i = i + 1;
         }
-        assertEquals(sequence.length, i, "Iterator yieled " + (i + 1) + " 
elements, but " +
+        assertEquals(sequence.length, i, "Iterator yielded " + (i + 1) + " 
elements, but " +
                 sequence.length + " were expected.");
     }
 
@@ -151,13 +151,13 @@ public class ImplicitLinkedHashCollectionTest {
         int i = 0;
         while (iter.hasNext()) {
             TestElement element = iter.next();
-            assertTrue(expectedIter.hasNext(), "Iterator yieled " + (i + 1) + 
" elements, but only " + i +
+            assertTrue(expectedIter.hasNext(), "Iterator yielded " + (i + 1) + 
" elements, but only " + i +
                 " were expected.");
             Integer expected = expectedIter.next();
             assertEquals(expected.intValue(), element.key, "Iterator value 
number " + (i + 1) + " was incorrect.");
             i = i + 1;
         }
-        assertFalse(expectedIter.hasNext(), "Iterator yieled " + i + " 
elements, but at least " + (i + 1) +
+        assertFalse(expectedIter.hasNext(), "Iterator yielded " + i + " 
elements, but at least " + (i + 1) +
             " were expected.");
     }
 
diff --git 
a/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollectionTest.java
 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashMultiCollectionTest.java
similarity index 92%
rename from 
clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollectionTest.java
rename to 
clients/src/test/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashMultiCollectionTest.java
index a04f23fa4ef..b8dc2b7d5de 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/utils/ImplicitLinkedHashMultiCollectionTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/ImplicitLinkedHashMultiCollectionTest.java
@@ -14,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.kafka.common.utils;
+package org.apache.kafka.common.utils.internals;
 
-import 
org.apache.kafka.common.utils.ImplicitLinkedHashCollectionTest.TestElement;
+import 
org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollectionTest.TestElement;
 
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
@@ -92,14 +92,14 @@ public class ImplicitLinkedHashMultiCollectionTest {
         int i = 0;
         while (iterator.hasNext()) {
             TestElement element = iterator.next();
-            assertTrue(i < sequence.length, "Iterator yieled " + (i + 1) + " 
elements, but only " +
+            assertTrue(i < sequence.length, "Iterator yielded " + (i + 1) + " 
elements, but only " +
                 sequence.length + " were expected.");
             if (sequence[i] != element) {
                 fail("Iterator value number " + (i + 1) + " was incorrect.");
             }
             i = i + 1;
         }
-        assertEquals(sequence.length, i, "Iterator yieled " + (i + 1) + " 
elements, but " +
+        assertEquals(sequence.length, i, "Iterator yielded " + (i + 1) + " 
elements, but " +
                 sequence.length + " were expected.");
     }
 
@@ -158,12 +158,12 @@ public class ImplicitLinkedHashMultiCollectionTest {
         while (iter.hasNext()) {
             TestElement element = iter.next();
             assertTrue(expectedIter.hasNext(),
-                "Iterator yieled " + (i + 1) + " elements, but only " + i + " 
were expected.");
+                "Iterator yielded " + (i + 1) + " elements, but only " + i + " 
were expected.");
             TestElement expected = expectedIter.next();
             assertSame(expected, element, "Iterator value number " + (i + 1) + 
" was incorrect.");
             i = i + 1;
         }
         assertFalse(expectedIter.hasNext(),
-            "Iterator yieled " + i + " elements, but at least " + (i + 1) + " 
were expected.");
+            "Iterator yielded " + i + " elements, but at least " + (i + 1) + " 
were expected.");
     }
 }
diff --git a/core/src/main/java/kafka/server/share/SharePartitionManager.java 
b/core/src/main/java/kafka/server/share/SharePartitionManager.java
index 27da4cdafa6..a6311c44bd8 100644
--- a/core/src/main/java/kafka/server/share/SharePartitionManager.java
+++ b/core/src/main/java/kafka/server/share/SharePartitionManager.java
@@ -29,8 +29,8 @@ import 
org.apache.kafka.common.message.ShareAcknowledgeResponseData;
 import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.requests.ShareRequestMetadata;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import 
org.apache.kafka.coordinator.group.modern.share.ShareGroupConfigProvider;
 import org.apache.kafka.server.common.ShareVersion;
 import org.apache.kafka.server.partition.PartitionListener;
diff --git 
a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java 
b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java
index 9039d8369ef..d4699b5dc09 100644
--- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java
+++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java
@@ -48,8 +48,8 @@ import org.apache.kafka.common.record.internal.SimpleRecord;
 import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.requests.ShareFetchResponse;
 import org.apache.kafka.common.requests.ShareRequestMetadata;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.coordinator.group.GroupConfigManager;
 import 
org.apache.kafka.coordinator.group.modern.share.ShareGroupConfigProvider;
 import org.apache.kafka.server.common.ShareVersion;
diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala 
b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
index cc4d9e377dd..186b8d1d260 100755
--- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
@@ -32,7 +32,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, 
ByteBufferAccessor, Errors}
 import org.apache.kafka.common.record.internal.{MemoryRecords, SimpleRecord}
 import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader}
 import org.apache.kafka.common.security.auth.SecurityProtocol
-import org.apache.kafka.common.utils.ByteUtils
+import org.apache.kafka.common.utils.internals.ByteUtils
 import org.apache.kafka.common.{TopicPartition, Uuid, requests}
 import org.apache.kafka.server.config.ServerLogConfigs
 import org.junit.jupiter.api.Assertions._
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala 
b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index a8277531bbc..c02431e0910 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -75,7 +75,8 @@ import org.apache.kafka.common.requests.{FetchMetadata => 
JFetchMetadata, _}
 import org.apache.kafka.common.resource.{PatternType, Resource, 
ResourcePattern, ResourceType}
 import org.apache.kafka.common.security.auth.{KafkaPrincipal, 
KafkaPrincipalSerde, SecurityProtocol}
 import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource
-import org.apache.kafka.common.utils.{ImplicitLinkedHashCollection, 
ProducerIdAndEpoch, SecurityUtils, Utils}
+import org.apache.kafka.common.utils.{ProducerIdAndEpoch, SecurityUtils, Utils}
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection
 import 
org.apache.kafka.coordinator.group.GroupConfig.{CONSUMER_ASSIGNMENT_INTERVAL_MS_CONFIG,
 CONSUMER_ASSIGNOR_OFFLOAD_ENABLE_CONFIG, 
CONSUMER_HEARTBEAT_INTERVAL_MS_CONFIG, CONSUMER_SESSION_TIMEOUT_MS_CONFIG, 
SHARE_ASSIGNMENT_INTERVAL_MS_CONFIG, SHARE_ASSIGNOR_OFFLOAD_ENABLE_CONFIG, 
SHARE_AUTO_OFFSET_RESET_CONFIG, SHARE_DELIVERY_COUNT_LIMIT_CONFIG, 
SHARE_HEARTBEAT_INTERVAL_MS_CONFIG, SHARE_ISOLATION_LEVEL_CONFIG, 
SHARE_PARTITION_MAX_RECORD_LOCKS_CONFIG, SHARE_RECORD_LOCK_DURATION_MS_CO [...]
 import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig
 import org.apache.kafka.coordinator.group.{GroupConfig, GroupConfigManager, 
GroupCoordinator, GroupCoordinatorConfig}
diff --git 
a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java 
b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java
index f235bfa17dc..f09baf230ab 100644
--- a/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java
+++ b/generator/src/main/java/org/apache/kafka/message/MessageGenerator.java
@@ -84,10 +84,10 @@ public final class MessageGenerator {
     static final String ARRAYLIST_CLASS = "java.util.ArrayList";
 
     static final String IMPLICIT_LINKED_HASH_COLLECTION_CLASS =
-        "org.apache.kafka.common.utils.ImplicitLinkedHashCollection";
+        "org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection";
 
     static final String IMPLICIT_LINKED_HASH_MULTI_COLLECTION_CLASS =
-        "org.apache.kafka.common.utils.ImplicitLinkedHashMultiCollection";
+        
"org.apache.kafka.common.utils.internals.ImplicitLinkedHashMultiCollection";
 
     static final String UNSUPPORTED_VERSION_EXCEPTION_CLASS =
         "org.apache.kafka.common.errors.UnsupportedVersionException";
@@ -120,7 +120,7 @@ public final class MessageGenerator {
 
     static final String RESPONSE_SUFFIX = "Response";
 
-    static final String BYTE_UTILS_CLASS = 
"org.apache.kafka.common.utils.ByteUtils";
+    static final String BYTE_UTILS_CLASS = 
"org.apache.kafka.common.utils.internals.ByteUtils";
 
     static final String STANDARD_CHARSETS = 
"java.nio.charset.StandardCharsets";
 
diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml
index 455c9814c78..6581d229ba6 100644
--- a/gradle/spotbugs-exclude.xml
+++ b/gradle/spotbugs-exclude.xml
@@ -678,7 +678,7 @@ For a detailed description of spotbugs bug categories, see 
https://spotbugs.read
     <Match>
         <Or>
             <!-- Internal comparator is not intended for serialization -->
-            <Class 
name="org.apache.kafka.common.utils.internals.BytesUtils$LexicographicByteArrayComparator"/>
+            <Class 
name="org.apache.kafka.common.utils.internals.ByteUtils$LexicographicByteArrayComparator"/>
             <!-- Benchmark comparator is not intended for serialization -->
             <Class 
name="org.apache.kafka.jmh.util.BytesCompareBenchmark$HandwrittenLexicoComparator"/>
         </Or>
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ImplicitLinkedHashCollectionBenchmark.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ImplicitLinkedHashCollectionBenchmark.java
index 8861428103a..0c41b0fc02b 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ImplicitLinkedHashCollectionBenchmark.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/ImplicitLinkedHashCollectionBenchmark.java
@@ -18,7 +18,7 @@
 package org.apache.kafka.jmh.common;
 
 import org.apache.kafka.common.Uuid;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/streams/RawBytesExtractionBenchmark.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/streams/RawBytesExtractionBenchmark.java
index 6fca76f18eb..09a47d2856e 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/streams/RawBytesExtractionBenchmark.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/streams/RawBytesExtractionBenchmark.java
@@ -20,7 +20,7 @@ package org.apache.kafka.jmh.streams;
 import org.apache.kafka.common.errors.SerializationException;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.LongDeserializer;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.state.HeadersBytesStore;
 import org.apache.kafka.streams.state.StateSerdes;
 import org.apache.kafka.streams.state.internals.Utils;
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ByteUtilsBenchmark.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ByteUtilsBenchmark.java
index be609087b23..1d0ee29edfe 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ByteUtilsBenchmark.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ByteUtilsBenchmark.java
@@ -17,7 +17,7 @@
 
 package org.apache.kafka.jmh.util;
 
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.CompilerControl;
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/BytesCompareBenchmark.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/BytesCompareBenchmark.java
index 23889576ca1..7fda479cc87 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/BytesCompareBenchmark.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/BytesCompareBenchmark.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.jmh.util;
 
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.Fork;
@@ -45,7 +45,7 @@ public class BytesCompareBenchmark {
 
     private byte[][] tv;
     private TreeMap<byte[], Integer> oldMap = new TreeMap<>(new 
HandwrittenLexicoComparator());
-    private TreeMap<byte[], Integer> newMap = new 
TreeMap<>(BytesUtils.BYTES_LEXICO_COMPARATOR);
+    private TreeMap<byte[], Integer> newMap = new 
TreeMap<>(ByteUtils.BYTES_LEXICO_COMPARATOR);
 
     @Setup
     public void setup() {
@@ -74,7 +74,7 @@ public class BytesCompareBenchmark {
         }
     }
 
-    static class HandwrittenLexicoComparator implements 
BytesUtils.ByteArrayComparator {
+    static class HandwrittenLexicoComparator implements 
ByteUtils.ByteArrayComparator {
         @Override
         public int compare(byte[] buffer1, byte[] buffer2) {
             return compare(buffer1, 0, buffer1.length, buffer2, 0, 
buffer2.length);
diff --git 
a/metadata/src/test/java/org/apache/kafka/metadata/MetadataRecordSerdeTest.java 
b/metadata/src/test/java/org/apache/kafka/metadata/MetadataRecordSerdeTest.java
index 1e61f10d8fe..16cb21ee289 100644
--- 
a/metadata/src/test/java/org/apache/kafka/metadata/MetadataRecordSerdeTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/metadata/MetadataRecordSerdeTest.java
@@ -21,7 +21,7 @@ import org.apache.kafka.common.metadata.RegisterBrokerRecord;
 import org.apache.kafka.common.metadata.TopicRecord;
 import org.apache.kafka.common.protocol.ByteBufferAccessor;
 import org.apache.kafka.common.protocol.ObjectSerializationCache;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.common.serialization.MetadataParseException;
 
diff --git 
a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java 
b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java
index bd15406cbb0..913174a6c07 100644
--- a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java
+++ b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java
@@ -23,7 +23,7 @@ import org.apache.kafka.common.metadata.TopicRecord;
 import org.apache.kafka.common.protocol.ApiMessage;
 import org.apache.kafka.common.protocol.Message;
 import org.apache.kafka.common.security.auth.SecurityProtocol;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
 import org.apache.kafka.server.common.MetadataVersion;
 import org.apache.kafka.server.util.MockRandom;
diff --git 
a/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java 
b/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java
index d31cb9a1c45..dda02eb72ae 100644
--- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java
+++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java
@@ -27,7 +27,7 @@ import 
org.apache.kafka.common.record.internal.DefaultRecordBatch;
 import org.apache.kafka.common.record.internal.MemoryRecords;
 import org.apache.kafka.common.record.internal.RecordBatch;
 import org.apache.kafka.common.utils.ByteBufferOutputStream;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.server.common.serialization.RecordSerde;
 
 import java.io.DataOutputStream;
diff --git 
a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java 
b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java
index b89439c6111..d18d6742ef6 100644
--- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java
+++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java
@@ -23,9 +23,9 @@ import org.apache.kafka.common.record.internal.MemoryRecords;
 import org.apache.kafka.common.record.internal.MutableRecordBatch;
 import org.apache.kafka.common.record.internal.Records;
 import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.raft.Batch;
 import org.apache.kafka.raft.ControlRecord;
 import org.apache.kafka.server.common.serialization.RecordSerde;
diff --git 
a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java 
b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java
index 7c5b59c4fa5..9b1694bfbe4 100644
--- 
a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java
+++ 
b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java
@@ -28,9 +28,9 @@ import 
org.apache.kafka.common.record.internal.ControlRecordUtils;
 import org.apache.kafka.common.record.internal.DefaultRecord;
 import org.apache.kafka.common.record.internal.MemoryRecordsBuilder;
 import org.apache.kafka.common.record.internal.RecordBatch;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.MockTime;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
diff --git 
a/server-common/src/main/java/org/apache/kafka/server/common/serialization/AbstractApiMessageSerde.java
 
b/server-common/src/main/java/org/apache/kafka/server/common/serialization/AbstractApiMessageSerde.java
index a29b6a00fc0..9651b1e3951 100644
--- 
a/server-common/src/main/java/org/apache/kafka/server/common/serialization/AbstractApiMessageSerde.java
+++ 
b/server-common/src/main/java/org/apache/kafka/server/common/serialization/AbstractApiMessageSerde.java
@@ -20,7 +20,7 @@ import org.apache.kafka.common.protocol.ApiMessage;
 import org.apache.kafka.common.protocol.ObjectSerializationCache;
 import org.apache.kafka.common.protocol.Readable;
 import org.apache.kafka.common.protocol.Writable;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.server.common.ApiMessageAndVersion;
 
 /**
diff --git a/server/src/main/java/org/apache/kafka/server/FetchContext.java 
b/server/src/main/java/org/apache/kafka/server/FetchContext.java
index 69c8dffb752..dc240a01174 100644
--- a/server/src/main/java/org/apache/kafka/server/FetchContext.java
+++ b/server/src/main/java/org/apache/kafka/server/FetchContext.java
@@ -25,8 +25,8 @@ import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.requests.FetchMetadata;
 import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.requests.FetchResponse;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.server.FetchSession.CachedPartition;
 import org.apache.kafka.server.FetchSession.FetchSessionCache;
 
diff --git a/server/src/main/java/org/apache/kafka/server/FetchSession.java 
b/server/src/main/java/org/apache/kafka/server/FetchSession.java
index 45ad79bb02f..db4af175c80 100644
--- a/server/src/main/java/org/apache/kafka/server/FetchSession.java
+++ b/server/src/main/java/org/apache/kafka/server/FetchSession.java
@@ -22,8 +22,8 @@ import org.apache.kafka.common.message.FetchResponseData;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.requests.FetchResponse;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.server.metrics.KafkaMetricsGroup;
 
 import java.util.ArrayList;
diff --git 
a/server/src/main/java/org/apache/kafka/server/FetchSessionCacheShard.java 
b/server/src/main/java/org/apache/kafka/server/FetchSessionCacheShard.java
index b2d808542ec..fd59b0ca1a3 100644
--- a/server/src/main/java/org/apache/kafka/server/FetchSessionCacheShard.java
+++ b/server/src/main/java/org/apache/kafka/server/FetchSessionCacheShard.java
@@ -17,8 +17,8 @@
 package org.apache.kafka.server;
 
 import org.apache.kafka.common.requests.FetchMetadata;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
 import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.server.FetchSession.EvictableKey;
 import org.apache.kafka.server.FetchSession.LastUsedKey;
 
diff --git 
a/server/src/main/java/org/apache/kafka/server/share/CachedSharePartition.java 
b/server/src/main/java/org/apache/kafka/server/share/CachedSharePartition.java
index 8cfc9365039..bafbf43b3a3 100644
--- 
a/server/src/main/java/org/apache/kafka/server/share/CachedSharePartition.java
+++ 
b/server/src/main/java/org/apache/kafka/server/share/CachedSharePartition.java
@@ -22,7 +22,7 @@ import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.message.ShareFetchResponseData;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.requests.ShareFetchResponse;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 
 import java.util.Objects;
 import java.util.Optional;
diff --git 
a/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java 
b/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java
index 7f9a47e29df..9df63c10b41 100644
--- 
a/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java
+++ 
b/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java
@@ -18,7 +18,7 @@
 package org.apache.kafka.server.share.session;
 
 import org.apache.kafka.common.TopicIdPartition;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.server.share.CachedSharePartition;
 
 import java.util.ArrayList;
diff --git 
a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java
 
b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java
index 01051698039..29a33146543 100644
--- 
a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java
+++ 
b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java
@@ -18,7 +18,7 @@
 package org.apache.kafka.server.share.session;
 
 import org.apache.kafka.common.requests.ShareRequestMetadata;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.server.metrics.KafkaMetricsGroup;
 import org.apache.kafka.server.network.ConnectionDisconnectListener;
 import org.apache.kafka.server.share.CachedSharePartition;
diff --git a/server/src/test/java/org/apache/kafka/server/FetchSessionTest.java 
b/server/src/test/java/org/apache/kafka/server/FetchSessionTest.java
index e929b789aac..a9308ed3b9b 100644
--- a/server/src/test/java/org/apache/kafka/server/FetchSessionTest.java
+++ b/server/src/test/java/org/apache/kafka/server/FetchSessionTest.java
@@ -28,7 +28,7 @@ import org.apache.kafka.common.requests.FetchMetadata;
 import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.requests.FetchRequest.PartitionData;
 import org.apache.kafka.common.requests.FetchResponse;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.server.FetchContext.FullFetchContext;
 import org.apache.kafka.server.FetchContext.IncrementalFetchContext;
 import org.apache.kafka.server.FetchContext.SessionErrorContext;
diff --git 
a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java
 
b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java
index 16da3016d29..c2cc3af450e 100644
--- 
a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java
+++ 
b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.server.share.session;
 
 import org.apache.kafka.common.Uuid;
-import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
+import org.apache.kafka.common.utils.internals.ImplicitLinkedHashCollection;
 import org.apache.kafka.server.share.CachedSharePartition;
 import org.apache.kafka.server.share.ShareGroupListener;
 import org.apache.kafka.test.TestUtils;
diff --git 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java
 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java
index 345021d3026..6f9f736b8ce 100644
--- 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java
+++ 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/ProducerStateManager.java
@@ -20,11 +20,11 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ByteBufferAccessor;
 import org.apache.kafka.common.protocol.MessageUtil;
 import org.apache.kafka.common.record.internal.RecordBatch;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Crc32C;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import 
org.apache.kafka.server.log.remote.metadata.storage.generated.ProducerSnapshot;
 
 import org.slf4j.Logger;
diff --git 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/SkimpyOffsetMap.java
 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/SkimpyOffsetMap.java
index 17800cc27ab..3e7fb195c7b 100644
--- 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/SkimpyOffsetMap.java
+++ 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/SkimpyOffsetMap.java
@@ -16,8 +16,8 @@
  */
 package org.apache.kafka.storage.internals.log;
 
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 import java.security.DigestException;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java
index 3ae46843289..6966ec12373 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java
@@ -18,7 +18,7 @@ package org.apache.kafka.streams.kstream.internals;
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Deserializer;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.processor.internals.SerdeGetter;
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java
index 219115c3b93..d7d88f71b9f 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java
@@ -18,7 +18,7 @@ package org.apache.kafka.streams.kstream.internals;
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.internals.UpgradeFromValues;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplier.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplier.java
index fd6038afda0..55660826efb 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplier.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignTableJoinProcessorSupplier.java
@@ -18,7 +18,7 @@ package 
org.apache.kafka.streams.kstream.internals.foreignkeyjoin;
 
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.kstream.internals.Change;
 import org.apache.kafka.streams.processor.api.ContextualProcessor;
@@ -124,7 +124,7 @@ public class ForeignTableJoinProcessorSupplier<KLeft, 
KRight, VRight>
 
             //Perform the prefixScan and propagate the results
             try (final KeyValueIterator<Bytes, 
ValueTimestampHeaders<SubscriptionWrapper<KLeft>>> prefixScanResults =
-                     subscriptionStore.range(prefixBytes, 
BytesUtils.increment(prefixBytes))) {
+                     subscriptionStore.range(prefixBytes, 
ByteUtils.increment(prefixBytes))) {
 
                 while (prefixScanResults.hasNext()) {
                     final KeyValue<Bytes, 
ValueTimestampHeaders<SubscriptionWrapper<KLeft>>> next = 
prefixScanResults.next();
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersSerializer.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersSerializer.java
index 4792052aa9e..9d982f6ba49 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersSerializer.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/AggregationWithHeadersSerializer.java
@@ -18,7 +18,7 @@ package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer;
 import org.apache.kafka.streams.processor.internals.SerdeGetter;
 import org.apache.kafka.streams.state.AggregationWithHeaders;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
index 3d6efc243a0..8ef68e0d06c 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
@@ -19,7 +19,7 @@ package org.apache.kafka.streams.state.internals;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.kstream.internals.Change;
 import org.apache.kafka.streams.processor.StateStore;
@@ -427,7 +427,7 @@ public class CachingKeyValueStore
         validateStoreOpen();
         final KeyValueIterator<Bytes, byte[]> storeIterator = 
wrapped().prefixScan(prefix, prefixKeySerializer);
         final Bytes from = Bytes.wrap(prefixKeySerializer.serialize(null, 
prefix));
-        final Bytes to = BytesUtils.increment(from);
+        final Bytes to = ByteUtils.increment(from);
         final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = 
internalContext.cache().range(cacheName, from, to, false);
         return new MergedSortedCacheKeyValueBytesStoreIterator(cacheIterator, 
storeIterator, true);
     }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/DualColumnFamilyAccessor.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/DualColumnFamilyAccessor.java
index 4d0ae0222ca..db4274cc625 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/DualColumnFamilyAccessor.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/DualColumnFamilyAccessor.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.ProcessorStateException;
 import org.apache.kafka.streams.state.internals.RocksDBStore.DBAccessor;
@@ -271,7 +271,7 @@ class DualColumnFamilyAccessor extends 
AbstractColumnFamilyAccessor {
         // RocksDB's JNI interface does not expose getters/setters that allow 
the
         // comparator to be pluggable, and the default is lexicographic, so 
it's
         // safe to just force lexicographic comparator here for now.
-        private final Comparator<byte[]> comparator = 
BytesUtils.BYTES_LEXICO_COMPARATOR;
+        private final Comparator<byte[]> comparator = 
ByteUtils.BYTES_LEXICO_COMPARATOR;
 
         private final String storeName;
         private final RocksIterator iterNewFormat;
@@ -401,7 +401,7 @@ class DualColumnFamilyAccessor extends 
AbstractColumnFamilyAccessor {
     // RocksDB's JNI interface does not expose getters/setters that allow the
     // comparator to be pluggable, and the default is lexicographic, so it's
     // safe to just force lexicographic comparator here for now.
-        private final Comparator<byte[]> comparator = 
BytesUtils.BYTES_LEXICO_COMPARATOR;
+        private final Comparator<byte[]> comparator = 
ByteUtils.BYTES_LEXICO_COMPARATOR;
         private final byte[] rawLastKey;
         private final boolean forward;
         private final boolean toInclusive;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersDeserializer.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersDeserializer.java
index 12ab040a68e..18090042ce8 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersDeserializer.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersDeserializer.java
@@ -18,7 +18,7 @@ package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.header.internals.RecordHeaders;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersSerializer.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersSerializer.java
index efadd05ba11..109ff9093d3 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersSerializer.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/HeadersSerializer.java
@@ -18,7 +18,7 @@ package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.Headers;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java
index f74d8e6387a..4b5336169bf 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java
@@ -20,7 +20,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.processor.StateStore;
@@ -174,7 +174,7 @@ public class InMemoryKeyValueStore implements 
KeyValueStore<Bytes, byte[]> {
     public synchronized <PS extends Serializer<P>, P> KeyValueIterator<Bytes, 
byte[]> prefixScan(final P prefix, final PS prefixKeySerializer) {
 
         final Bytes from = Bytes.wrap(prefixKeySerializer.serialize(null, 
prefix));
-        final Bytes to = BytesUtils.increment(from);
+        final Bytes to = ByteUtils.increment(from);
 
         return new InMemoryKeyValueIterator(map.subMap(from, true, to, 
false).keySet(), true);
     }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java
index 1e5e102c0a1..0406f217aec 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryNavigableLRUCache.java
@@ -18,7 +18,7 @@ package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.query.PositionBound;
 import org.apache.kafka.streams.query.Query;
@@ -90,7 +90,7 @@ public class MemoryNavigableLRUCache extends MemoryLRUCache {
     public <PS extends Serializer<P>, P> KeyValueIterator<Bytes, byte[]> 
prefixScan(final P prefix, final PS prefixKeySerializer) {
 
         final Bytes from = Bytes.wrap(prefixKeySerializer.serialize(null, 
prefix));
-        final Bytes to = BytesUtils.increment(from);
+        final Bytes to = ByteUtils.increment(from);
 
         final TreeMap<Bytes, byte[]> treeMap = toTreeMap();
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java
index 058edad52c8..c040e831f8b 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java
@@ -18,7 +18,7 @@ package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.header.Headers;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import java.nio.ByteBuffer;
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java
index a8f7c6fcf79..0617349c724 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBRangeIterator.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 
 import org.rocksdb.RocksIterator;
@@ -28,7 +28,7 @@ class RocksDBRangeIterator extends RocksDbIterator {
     // RocksDB's JNI interface does not expose getters/setters that allow the
     // comparator to be pluggable, and the default is lexicographic, so it's
     // safe to just force lexicographic comparator here for now.
-    private final Comparator<byte[]> comparator = 
BytesUtils.BYTES_LEXICO_COMPARATOR;
+    private final Comparator<byte[]> comparator = 
ByteUtils.BYTES_LEXICO_COMPARATOR;
     private final byte[] rawLastKey;
     private final boolean forward;
     private final boolean toInclusive;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
index 783cae5b8c2..621e1eaa566 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
@@ -22,7 +22,7 @@ import org.apache.kafka.common.metrics.Sensor.RecordingLevel;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Utils;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.errors.InvalidStateStoreException;
@@ -579,7 +579,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, 
byte[]>, BatchWritingS
         // RocksDB's deleteRange() does not support a null upper bound so in 
the event
         // of overflow from increment(), the operation cannot be performed and 
an
         // IndexOutOfBoundsException will be thrown.
-        cfAccessor.deleteRange(dbAccessor, keyFrom.get(), 
BytesUtils.increment(keyTo).get());
+        cfAccessor.deleteRange(dbAccessor, keyFrom.get(), 
ByteUtils.increment(keyTo).get());
     }
 
     @Override
@@ -1168,7 +1168,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, 
byte[]>, BatchWritingS
     }
 
     /**
-     * Same as {@link BytesUtils#increment(Bytes)} but {@code null} is 
returned instead of throwing
+     * Same as {@link ByteUtils#increment(Bytes)} but {@code null} is returned 
instead of throwing
      * {@code IndexOutOfBoundsException} in the event of overflow.
      *
      * @param input bytes to increment
@@ -1177,7 +1177,7 @@ public class RocksDBStore implements KeyValueStore<Bytes, 
byte[]>, BatchWritingS
      */
     static Bytes incrementWithoutOverflow(final Bytes input) {
         try {
-            return BytesUtils.increment(input);
+            return ByteUtils.increment(input);
         } catch (final IndexOutOfBoundsException e) {
             return null;
         }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunction.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunction.java
index 1d3cbea4a9f..089c6801269 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunction.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedCacheFunction.java
@@ -18,7 +18,7 @@
 package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.state.internals.SegmentedBytesStore.KeySchema;
 
 import java.nio.ByteBuffer;
@@ -78,7 +78,7 @@ class SegmentedCacheFunction implements CacheFunction {
         if (segmentCompare == 0) {
             final byte[] cacheKeyBytes = cacheKey.get();
             final byte[] storeKeyBytes = storeKey.get();
-            return BytesUtils.BYTES_LEXICO_COMPARATOR.compare(
+            return ByteUtils.BYTES_LEXICO_COMPARATOR.compare(
                 cacheKeyBytes, SEGMENT_ID_BYTES, cacheKeyBytes.length - 
SEGMENT_ID_BYTES,
                 storeKeyBytes, 0, storeKeyBytes.length
             );
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/Utils.java 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/Utils.java
index c67bfc8dfbe..6e9ad91f810 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Utils.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Utils.java
@@ -20,8 +20,8 @@ import org.apache.kafka.common.errors.SerializationException;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.header.internals.RecordHeaders;
 import org.apache.kafka.common.serialization.LongDeserializer;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.state.StateSerdes;
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java
index a0f9acc4c9b..73f41be794c 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersDeserializer.java
@@ -19,7 +19,7 @@ package org.apache.kafka.streams.state.internals;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.LongDeserializer;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer;
 import org.apache.kafka.streams.processor.internals.SerdeGetter;
 import org.apache.kafka.streams.state.ValueTimestampHeaders;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializer.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializer.java
index f4e6fe07a22..df9dcba74ee 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializer.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ValueTimestampHeadersSerializer.java
@@ -18,7 +18,7 @@ package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer;
 import org.apache.kafka.streams.processor.internals.SerdeGetter;
 import org.apache.kafka.streams.state.ValueTimestampHeaders;
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/ChangedSerdeTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/ChangedSerdeTest.java
index 4d1b2b0b212..50dea706aeb 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/ChangedSerdeTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/ChangedSerdeTest.java
@@ -19,7 +19,7 @@ package org.apache.kafka.streams.kstream.internals;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.Serializer;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.errors.StreamsException;
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/HeadersBytesStoreTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/HeadersBytesStoreTest.java
index 37cacde22c0..d597b97daed 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/HeadersBytesStoreTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/HeadersBytesStoreTest.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.streams.state;
 
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.junit.jupiter.api.Test;
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java
index 05d64e31ed6..504b7b3e07c 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java
@@ -25,7 +25,7 @@ import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.common.utils.MockTime;
 import org.apache.kafka.common.utils.Time;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.StreamsConfig.InternalConfig;
@@ -221,7 +221,7 @@ public class ChangeLoggingKeyValueBytesStoreTest {
     @Test
     public void shouldGetRecordsWithPrefixKey() {
         store.put(hi, there);
-        store.put(BytesUtils.increment(hi), world);
+        store.put(ByteUtils.increment(hi), world);
 
         final List<Bytes> keys = new ArrayList<>();
         final List<Bytes> values = new ArrayList<>();
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBRangeIteratorTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBRangeIteratorTest.java
index 8ef4046794d..0ed49a3c75c 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBRangeIteratorTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBRangeIteratorTest.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.internals.BytesUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
@@ -125,7 +125,7 @@ public class RocksDBRangeIteratorTest {
 
     @Test
     public void 
shouldReturnAllKeysWhenLastKeyIsGreaterThanLargestKeyInStateStoreInForwardDirection()
 {
-        final Bytes toBytes = BytesUtils.increment(key4Bytes);
+        final Bytes toBytes = ByteUtils.increment(key4Bytes);
         final RocksIterator rocksIterator = mock(RocksIterator.class);
         doNothing().when(rocksIterator).seek(key1Bytes.get());
         when(rocksIterator.isValid())
@@ -341,7 +341,7 @@ public class RocksDBRangeIteratorTest {
     @Test
     public void 
shouldReturnTheCurrentKeyOnInvokingPeekNextKeyInReverseDirection() {
         final RocksIterator rocksIterator = mock(RocksIterator.class);
-        final Bytes toBytes = BytesUtils.increment(key4Bytes);
+        final Bytes toBytes = ByteUtils.increment(key4Bytes);
         doNothing().when(rocksIterator).seekForPrev(toBytes.get());
         when(rocksIterator.isValid())
             .thenReturn(true)
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedSessionStoreUpgradeTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedSessionStoreUpgradeTest.java
index 260fe0f325d..e88ad164b30 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedSessionStoreUpgradeTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedSessionStoreUpgradeTest.java
@@ -20,8 +20,8 @@ import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.header.internals.RecordHeaders;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringSerializer;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.DslStoreFormat;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreUpgradeTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreUpgradeTest.java
index 9a216ee2b2d..12829a3b995 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreUpgradeTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreUpgradeTest.java
@@ -19,8 +19,8 @@ package org.apache.kafka.streams.state.internals;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.header.internals.RecordHeaders;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.ByteUtils;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.state.WindowStore;
 import org.apache.kafka.test.InternalMockProcessorContext;
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/UtilsTest.java 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/UtilsTest.java
index 631f906ee83..51732bd8283 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/UtilsTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/UtilsTest.java
@@ -20,7 +20,7 @@ import org.apache.kafka.common.errors.SerializationException;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.header.internals.RecordHeaders;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.ByteUtils;
+import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.state.StateSerdes;
 import org.apache.kafka.streams.state.ValueAndTimestamp;
 import org.apache.kafka.streams.state.ValueTimestampHeaders;


Reply via email to