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

payang 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 2299a06e319 KAFKA-20297: Move AbstractIterator, CircularIterator, 
CloseableIterator... into internal (#22052)
2299a06e319 is described below

commit 2299a06e3199b82969e42d91eab7c40f6a45a0e8
Author: Ken Huang <[email protected]>
AuthorDate: Wed Apr 15 20:53:15 2026 +0800

    KAFKA-20297: Move AbstractIterator, CircularIterator, CloseableIterator... 
into internal (#22052)
    
    We have exposed several classes in org.apache.kafka.common.utils as
    public APIs, which can be misleading since not all classes in this
    package are intended for external use.
    
    To clarify API boundaries and prevent unintended usage, we propose
    moving the following classes to an internal package:
    
    - AbstractIterator
    - CircularIterator
    - CloseableIterator
    - CopyOnWriteMap
    - FixedOrderMap
    - PrimitiveRef
    
    Reviewers: Christo <[email protected]>, PoAn Yang <[email protected]>
---
 .../org/apache/kafka/clients/consumer/ConsumerRecords.java     |  2 +-
 .../org/apache/kafka/clients/consumer/RoundRobinAssignor.java  |  2 +-
 .../kafka/clients/consumer/internals/CompletedFetch.java       |  2 +-
 .../kafka/clients/consumer/internals/ShareCompletedFetch.java  |  2 +-
 .../kafka/clients/producer/internals/RecordAccumulator.java    |  2 +-
 .../kafka/clients/producer/internals/TxnPartitionEntry.java    |  2 +-
 .../apache/kafka/common/header/internals/RecordHeaders.java    |  2 +-
 .../common/record/internal/AbstractLegacyRecordBatch.java      |  4 ++--
 .../apache/kafka/common/record/internal/AbstractRecords.java   |  2 +-
 .../kafka/common/record/internal/DefaultRecordBatch.java       |  2 +-
 .../kafka/common/record/internal/FileLogInputStream.java       |  2 +-
 .../org/apache/kafka/common/record/internal/FileRecords.java   |  2 +-
 .../org/apache/kafka/common/record/internal/MemoryRecords.java |  4 ++--
 .../kafka/common/record/internal/MutableRecordBatch.java       |  2 +-
 .../org/apache/kafka/common/record/internal/RecordBatch.java   |  2 +-
 .../kafka/common/record/internal/RecordBatchIterator.java      |  2 +-
 .../java/org/apache/kafka/common/record/internal/Records.java  |  2 +-
 .../kafka/common/utils/{ => internals}/AbstractIterator.java   |  2 +-
 .../kafka/common/utils/{ => internals}/CircularIterator.java   |  2 +-
 .../kafka/common/utils/{ => internals}/CloseableIterator.java  |  2 +-
 .../kafka/common/utils/{ => internals}/CopyOnWriteMap.java     |  2 +-
 .../kafka/common/utils/{ => internals}/FixedOrderMap.java      |  2 +-
 .../kafka/common/utils/{ => internals}/PrimitiveRef.java       |  2 +-
 .../kafka/common/record/internal/DefaultRecordBatchTest.java   |  2 +-
 .../kafka/common/record/internal/MemoryRecordsBuilderTest.java |  2 +-
 .../apache/kafka/common/record/internal/MemoryRecordsTest.java |  2 +-
 .../common/utils/{ => internals}/AbstractIteratorTest.java     |  2 +-
 .../common/utils/{ => internals}/CircularIteratorTest.java     |  2 +-
 .../kafka/common/utils/{ => internals}/FixedOrderMapTest.java  |  2 +-
 .../kafka/common/utils/{ => internals}/PrimitiveRefTest.java   |  2 +-
 .../java/org/apache/kafka/connect/header/ConnectHeaders.java   |  2 +-
 .../kafka/connect/runtime/distributed/EagerAssignor.java       |  2 +-
 gradle/spotbugs-exclude.xml                                    |  2 +-
 .../java/org/apache/kafka/jmh/log/TestLinearWriteSpeed.java    |  2 +-
 .../jmh/record/CompressedRecordBatchValidationBenchmark.java   |  2 +-
 .../apache/kafka/jmh/record/RecordBatchIterationBenchmark.java |  2 +-
 .../jmh/record/UncompressedRecordBatchValidationBenchmark.java |  2 +-
 .../java/org/apache/kafka/jmh/util/ConcurrentMapBenchmark.java |  2 +-
 .../kafka/server/log/remote/storage/RemoteLogManager.java      | 10 +++++-----
 .../java/org/apache/kafka/storage/internals/log/Cleaner.java   |  2 +-
 .../org/apache/kafka/storage/internals/log/LogValidator.java   |  6 +++---
 .../org/apache/kafka/storage/internals/log/UnifiedLog.java     |  2 +-
 .../apache/kafka/storage/internals/log/LogValidatorTest.java   |  2 +-
 .../streams/processor/internals/GlobalStateManagerImpl.java    |  2 +-
 .../streams/processor/internals/ProcessorStateManager.java     |  2 +-
 .../streams/state/internals/DualColumnFamilyAccessor.java      |  3 ++-
 .../apache/kafka/streams/state/internals/ListValueStore.java   |  2 +-
 .../apache/kafka/streams/state/internals/RocksDbIterator.java  |  2 +-
 .../org/apache/kafka/streams/state/internals/ThreadCache.java  |  2 +-
 49 files changed, 58 insertions(+), 57 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java 
b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
index 4f179492d2e..42fdad92312 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
@@ -17,7 +17,7 @@
 package org.apache.kafka.clients.consumer;
 
 import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 
 import java.util.ArrayList;
 import java.util.Collections;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
index 2d6edeaca07..7dc3a293c6c 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
@@ -18,8 +18,8 @@ package org.apache.kafka.clients.consumer;
 
 import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor;
 import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.utils.CircularIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.CircularIterator;
 
 import java.util.ArrayList;
 import java.util.HashMap;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java
index f6fc3beeb84..7eae356e8a5 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CompletedFetch.java
@@ -35,7 +35,7 @@ import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.requests.FetchResponse;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.common.utils.CloseableIterator;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 import org.slf4j.Logger;
 
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareCompletedFetch.java
 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareCompletedFetch.java
index 7a6c2291c93..f7fe9901b8e 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareCompletedFetch.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareCompletedFetch.java
@@ -34,8 +34,8 @@ import org.apache.kafka.common.requests.ShareFetchRequest;
 import org.apache.kafka.common.requests.ShareFetchResponse;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.common.utils.CloseableIterator;
 import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 import org.slf4j.Logger;
 
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
 
b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index 19c9728f011..81222f5cb88 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -35,10 +35,10 @@ import 
org.apache.kafka.common.record.internal.MemoryRecords;
 import org.apache.kafka.common.record.internal.MemoryRecordsBuilder;
 import org.apache.kafka.common.record.internal.Record;
 import org.apache.kafka.common.record.internal.RecordBatch;
-import org.apache.kafka.common.utils.CopyOnWriteMap;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.common.utils.ProducerIdAndEpoch;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.CopyOnWriteMap;
 import org.apache.kafka.common.utils.internals.ExponentialBackoff;
 
 import org.slf4j.Logger;
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionEntry.java
 
b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionEntry.java
index 1c3699eba46..65905e1e14a 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionEntry.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TxnPartitionEntry.java
@@ -20,8 +20,8 @@ package org.apache.kafka.clients.producer.internals;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.record.internal.DefaultRecordBatch;
 import org.apache.kafka.common.requests.ProduceResponse;
-import org.apache.kafka.common.utils.PrimitiveRef;
 import org.apache.kafka.common.utils.ProducerIdAndEpoch;
+import org.apache.kafka.common.utils.internals.PrimitiveRef;
 
 import java.util.Comparator;
 import java.util.OptionalInt;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java
 
b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java
index 3c7dabeb667..543a23ceea3 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java
@@ -19,7 +19,7 @@ package org.apache.kafka.common.header.internals;
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.record.internal.Record;
-import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 
 import java.util.ArrayList;
 import java.util.Arrays;
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 7738f4059d8..13cb169bcc1 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
@@ -22,12 +22,12 @@ import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.errors.CorruptRecordException;
 import org.apache.kafka.common.header.Header;
 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.CloseableIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 import org.apache.kafka.common.utils.internals.ByteUtils;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/AbstractRecords.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/AbstractRecords.java
index ad57ab45372..d0704d18ec3 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/AbstractRecords.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/AbstractRecords.java
@@ -17,8 +17,8 @@
 package org.apache.kafka.common.record.internal;
 
 import org.apache.kafka.common.header.Header;
-import org.apache.kafka.common.utils.AbstractIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 
 import java.nio.ByteBuffer;
 import java.util.Iterator;
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 2b6bc7ee11f..b62e9dc3301 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.CloseableIterator;
 import org.apache.kafka.common.utils.Crc32C;
 import org.apache.kafka.common.utils.internals.ByteUtils;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 import java.io.IOException;
 import java.io.InputStream;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/FileLogInputStream.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/FileLogInputStream.java
index 1d308d53af3..493607f1837 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/FileLogInputStream.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/FileLogInputStream.java
@@ -22,8 +22,8 @@ import org.apache.kafka.common.record.TimestampType;
 import 
org.apache.kafka.common.record.internal.AbstractLegacyRecordBatch.LegacyFileChannelRecordBatch;
 import 
org.apache.kafka.common.record.internal.DefaultRecordBatch.DefaultFileChannelRecordBatch;
 import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.common.utils.CloseableIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/FileRecords.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/FileRecords.java
index e0303a9c9ea..f62ecd3c2b4 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/FileRecords.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/FileRecords.java
@@ -19,8 +19,8 @@ package org.apache.kafka.common.record.internal;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.network.TransferableChannel;
 import 
org.apache.kafka.common.record.internal.FileLogInputStream.FileChannelRecordBatch;
-import org.apache.kafka.common.utils.AbstractIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 
 import java.io.Closeable;
 import java.io.File;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/MemoryRecords.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/MemoryRecords.java
index 5d3d0a2dbec..3a6ee63bb4d 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/MemoryRecords.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/MemoryRecords.java
@@ -27,11 +27,11 @@ import org.apache.kafka.common.network.TransferableChannel;
 import org.apache.kafka.common.record.TimestampType;
 import 
org.apache.kafka.common.record.internal.MemoryRecords.RecordFilter.BatchRetention;
 import 
org.apache.kafka.common.record.internal.MemoryRecords.RecordFilter.BatchRetentionResult;
-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.CloseableIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/MutableRecordBatch.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/MutableRecordBatch.java
index eedcf3eea21..cdb0ed22efd 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/MutableRecordBatch.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/MutableRecordBatch.java
@@ -19,7 +19,7 @@ package org.apache.kafka.common.record.internal;
 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.CloseableIterator;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 /**
  * A mutable record batch is one that can be modified in place (without 
copying). This is used by the broker
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/RecordBatch.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/RecordBatch.java
index 9bf42a5dc8d..96e83270259 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/RecordBatch.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/RecordBatch.java
@@ -18,7 +18,7 @@ package org.apache.kafka.common.record.internal;
 
 import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.common.utils.CloseableIterator;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 import java.nio.ByteBuffer;
 import java.util.Iterator;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/RecordBatchIterator.java
 
b/clients/src/main/java/org/apache/kafka/common/record/internal/RecordBatchIterator.java
index 707ee96d247..be882e02891 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/record/internal/RecordBatchIterator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/record/internal/RecordBatchIterator.java
@@ -18,7 +18,7 @@ package org.apache.kafka.common.record.internal;
 
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.errors.CorruptRecordException;
-import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 
 import java.io.EOFException;
 import java.io.IOException;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/record/internal/Records.java 
b/clients/src/main/java/org/apache/kafka/common/record/internal/Records.java
index ff77cd67f2f..3264698b55a 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/internal/Records.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/internal/Records.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.common.record.internal;
 
-import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 
 import java.util.Iterator;
 import java.util.Optional;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/AbstractIterator.java
similarity index 98%
rename from 
clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/AbstractIterator.java
index daf89e67821..60f6ad21a52 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/AbstractIterator.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 java.util.Iterator;
 import java.util.NoSuchElementException;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/CircularIterator.java 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/CircularIterator.java
similarity index 98%
rename from 
clients/src/main/java/org/apache/kafka/common/utils/CircularIterator.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/CircularIterator.java
index 925f4adf9e9..e3c95b13253 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/CircularIterator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/CircularIterator.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.Collection;
 import java.util.ConcurrentModificationException;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/CloseableIterator.java 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/CloseableIterator.java
similarity index 97%
rename from 
clients/src/main/java/org/apache/kafka/common/utils/CloseableIterator.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/CloseableIterator.java
index 1709b5d47fb..13dccc477d5 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/CloseableIterator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/CloseableIterator.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 java.io.Closeable;
 import java.util.Iterator;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/CopyOnWriteMap.java
similarity index 98%
rename from 
clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/CopyOnWriteMap.java
index 3283db209e2..2e7bd69ff8c 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/CopyOnWriteMap.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 java.util.Collection;
 import java.util.Collections;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/FixedOrderMap.java 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/FixedOrderMap.java
similarity index 97%
rename from 
clients/src/main/java/org/apache/kafka/common/utils/FixedOrderMap.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/FixedOrderMap.java
index 175282e7e09..fd75f3d7db1 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/FixedOrderMap.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/FixedOrderMap.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 java.util.LinkedHashMap;
 import java.util.Map;
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/PrimitiveRef.java 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/PrimitiveRef.java
similarity index 97%
rename from 
clients/src/main/java/org/apache/kafka/common/utils/PrimitiveRef.java
rename to 
clients/src/main/java/org/apache/kafka/common/utils/internals/PrimitiveRef.java
index d41fad2f98e..e5c95483fee 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/PrimitiveRef.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/utils/internals/PrimitiveRef.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;
 
 /**
  * Primitive reference used to pass primitive typed values as 
parameter-by-reference.
diff --git 
a/clients/src/test/java/org/apache/kafka/common/record/internal/DefaultRecordBatchTest.java
 
b/clients/src/test/java/org/apache/kafka/common/record/internal/DefaultRecordBatchTest.java
index feb5ce82695..d960a4b8261 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/record/internal/DefaultRecordBatchTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/record/internal/DefaultRecordBatchTest.java
@@ -25,8 +25,8 @@ import org.apache.kafka.common.header.internals.RecordHeader;
 import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.BufferSupplier;
 import org.apache.kafka.common.utils.ChunkedBytesStream;
-import org.apache.kafka.common.utils.CloseableIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 import org.apache.kafka.test.TestUtils;
 
 import org.junit.jupiter.api.Test;
diff --git 
a/clients/src/test/java/org/apache/kafka/common/record/internal/MemoryRecordsBuilderTest.java
 
b/clients/src/test/java/org/apache/kafka/common/record/internal/MemoryRecordsBuilderTest.java
index 2bceaec415d..8e6e503dd4d 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/record/internal/MemoryRecordsBuilderTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/record/internal/MemoryRecordsBuilderTest.java
@@ -22,8 +22,8 @@ import 
org.apache.kafka.common.message.LeaderChangeMessage.Voter;
 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.CloseableIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 import org.apache.kafka.test.TestUtils;
 
 import org.junit.jupiter.api.Test;
diff --git 
a/clients/src/test/java/org/apache/kafka/common/record/internal/MemoryRecordsTest.java
 
b/clients/src/test/java/org/apache/kafka/common/record/internal/MemoryRecordsTest.java
index c4f95f07baa..c5228b17acf 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/record/internal/MemoryRecordsTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/record/internal/MemoryRecordsTest.java
@@ -25,8 +25,8 @@ import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.record.internal.MemoryRecords.RecordFilter;
 import 
org.apache.kafka.common.record.internal.MemoryRecords.RecordFilter.BatchRetention;
 import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.common.utils.CloseableIterator;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 import org.apache.kafka.test.TestUtils;
 
 import org.junit.jupiter.api.Test;
diff --git 
a/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/AbstractIteratorTest.java
similarity index 97%
rename from 
clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java
rename to 
clients/src/test/java/org/apache/kafka/common/utils/internals/AbstractIteratorTest.java
index abc3921ccbe..760db7e8d75 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/AbstractIteratorTest.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;
 
diff --git 
a/clients/src/test/java/org/apache/kafka/common/utils/CircularIteratorTest.java 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/CircularIteratorTest.java
similarity index 97%
rename from 
clients/src/test/java/org/apache/kafka/common/utils/CircularIteratorTest.java
rename to 
clients/src/test/java/org/apache/kafka/common/utils/internals/CircularIteratorTest.java
index c5551bc5189..1eea9f8e62b 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/utils/CircularIteratorTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/CircularIteratorTest.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.kafka.common.utils;
+package org.apache.kafka.common.utils.internals;
 
 import org.junit.jupiter.api.Test;
 
diff --git 
a/clients/src/test/java/org/apache/kafka/common/utils/FixedOrderMapTest.java 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/FixedOrderMapTest.java
similarity index 97%
rename from 
clients/src/test/java/org/apache/kafka/common/utils/FixedOrderMapTest.java
rename to 
clients/src/test/java/org/apache/kafka/common/utils/internals/FixedOrderMapTest.java
index 4365de3d497..9d4ad6d2871 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/FixedOrderMapTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/FixedOrderMapTest.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;
 
diff --git 
a/clients/src/test/java/org/apache/kafka/common/utils/PrimitiveRefTest.java 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/PrimitiveRefTest.java
similarity index 96%
rename from 
clients/src/test/java/org/apache/kafka/common/utils/PrimitiveRefTest.java
rename to 
clients/src/test/java/org/apache/kafka/common/utils/internals/PrimitiveRefTest.java
index 1242136afd5..8876f2f6dd1 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/PrimitiveRefTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/utils/internals/PrimitiveRefTest.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;
 
diff --git 
a/connect/api/src/main/java/org/apache/kafka/connect/header/ConnectHeaders.java 
b/connect/api/src/main/java/org/apache/kafka/connect/header/ConnectHeaders.java
index 63ee8ab6598..19afc137c32 100644
--- 
a/connect/api/src/main/java/org/apache/kafka/connect/header/ConnectHeaders.java
+++ 
b/connect/api/src/main/java/org/apache/kafka/connect/header/ConnectHeaders.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.connect.header;
 
-import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 import org.apache.kafka.connect.data.Date;
 import org.apache.kafka.connect.data.Decimal;
 import org.apache.kafka.connect.data.Schema;
diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/EagerAssignor.java
 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/EagerAssignor.java
index 1004382e102..68ad83ef6b6 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/EagerAssignor.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/EagerAssignor.java
@@ -16,8 +16,8 @@
  */
 package org.apache.kafka.connect.runtime.distributed;
 
-import org.apache.kafka.common.utils.CircularIterator;
 import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.internals.CircularIterator;
 import org.apache.kafka.connect.storage.ClusterConfigState;
 import org.apache.kafka.connect.util.ConnectorTaskId;
 
diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml
index c4818c76c67..664777decf6 100644
--- a/gradle/spotbugs-exclude.xml
+++ b/gradle/spotbugs-exclude.xml
@@ -430,7 +430,7 @@ For a detailed description of spotbugs bug categories, see 
https://spotbugs.read
 
     <Match>
         <!-- False positive - the field is used in other modules and its unit 
test -->
-        <Class name="org.apache.kafka.common.utils.PrimitiveRef$LongRef"/>
+        <Class 
name="org.apache.kafka.common.utils.internals.PrimitiveRef$LongRef"/>
         <Field name="value"/>
         <Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD"/>
     </Match>
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/log/TestLinearWriteSpeed.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/log/TestLinearWriteSpeed.java
index 38914832cdb..a3e4510b70c 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/log/TestLinearWriteSpeed.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/log/TestLinearWriteSpeed.java
@@ -25,10 +25,10 @@ import 
org.apache.kafka.common.record.internal.CompressionType;
 import org.apache.kafka.common.record.internal.MemoryRecords;
 import org.apache.kafka.common.record.internal.Records;
 import org.apache.kafka.common.record.internal.SimpleRecord;
-import org.apache.kafka.common.utils.CopyOnWriteMap;
 import org.apache.kafka.common.utils.Exit;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.CopyOnWriteMap;
 import org.apache.kafka.coordinator.transaction.TransactionLogConfig;
 import org.apache.kafka.server.common.RequestLocal;
 import org.apache.kafka.server.util.CommandLineUtils;
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java
index ee8e3eef32a..0584ef67245 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java
@@ -21,8 +21,8 @@ import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.record.internal.CompressionType;
 import org.apache.kafka.common.record.internal.MemoryRecords;
-import org.apache.kafka.common.utils.PrimitiveRef;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.PrimitiveRef;
 import org.apache.kafka.storage.internals.log.AppendOrigin;
 import org.apache.kafka.storage.internals.log.LogValidator;
 
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java
index 90e4325e6b1..568f4be030e 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java
@@ -22,7 +22,7 @@ import org.apache.kafka.common.record.internal.MemoryRecords;
 import org.apache.kafka.common.record.internal.MutableRecordBatch;
 import org.apache.kafka.common.record.internal.Record;
 import org.apache.kafka.common.record.internal.RecordBatch;
-import org.apache.kafka.common.utils.CloseableIterator;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.Fork;
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java
index 11a806eee78..2b25b6bf1a2 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java
@@ -21,8 +21,8 @@ import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.record.internal.CompressionType;
 import org.apache.kafka.common.record.internal.MemoryRecords;
-import org.apache.kafka.common.utils.PrimitiveRef;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.PrimitiveRef;
 import org.apache.kafka.storage.internals.log.AppendOrigin;
 import org.apache.kafka.storage.internals.log.LogValidator;
 
diff --git 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ConcurrentMapBenchmark.java
 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ConcurrentMapBenchmark.java
index 7d7716cae2e..ca377c142d3 100644
--- 
a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ConcurrentMapBenchmark.java
+++ 
b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/util/ConcurrentMapBenchmark.java
@@ -17,7 +17,7 @@
 
 package org.apache.kafka.jmh.util;
 
-import org.apache.kafka.common.utils.CopyOnWriteMap;
+import org.apache.kafka.common.utils.internals.CopyOnWriteMap;
 
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
diff --git 
a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java
 
b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java
index 4f56f95c0b2..178d0307141 100644
--- 
a/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java
+++ 
b/storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java
@@ -39,10 +39,10 @@ import 
org.apache.kafka.common.record.internal.RemoteLogInputStream;
 import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.utils.BufferSupplier;
 import org.apache.kafka.common.utils.ChildFirstClassLoader;
-import org.apache.kafka.common.utils.CloseableIterator;
 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.CloseableIterator;
 import org.apache.kafka.common.utils.internals.ThreadUtils;
 import org.apache.kafka.server.common.CheckpointFile;
 import org.apache.kafka.server.common.OffsetAndEpoch;
@@ -1049,7 +1049,7 @@ public class RemoteLogManager implements Closeable, 
AsyncOffsetReader {
             
brokerTopicStats.topicStats(log.topicPartition().topic()).remoteCopyRequestRate().mark();
             brokerTopicStats.allTopicsStats().remoteCopyRequestRate().mark();
             Optional<CustomMetadata> customMetadata;
-            
+
             try {
                 customMetadata = 
remoteStorageManagerPlugin.get().copyLogSegmentData(copySegmentStartedRlsm, 
segmentData);
             } catch (RetriableRemoteStorageException e) {
@@ -1284,7 +1284,7 @@ public class RemoteLogManager implements Closeable, 
AsyncOffsetReader {
             private boolean 
deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry,
                                                                              
RemoteLogSegmentMetadata metadata)
                     throws RemoteStorageException, ExecutionException, 
InterruptedException {
-                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, 
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
                     ignored -> 
metadata.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < 
earliestEpochEntry.epoch()));
                 if (isSegmentDeleted) {
                     logger.info("Deleted remote log segment {} due to 
leader-epoch-cache truncation. " +
@@ -1565,7 +1565,7 @@ public class RemoteLogManager implements Closeable, 
AsyncOffsetReader {
                     Iterator<RemoteLogSegmentMetadata> segmentsIterator = 
remoteLogMetadataManagerPlugin.get().listRemoteLogSegments(topicIdPartition, 
epoch);
                     while (segmentsIterator.hasNext()) {
                         RemoteLogSegmentMetadata segmentMetadata = 
segmentsIterator.next();
-                        // Count only the size of segments in 
"COPY_SEGMENT_FINISHED" state because 
+                        // Count only the size of segments in 
"COPY_SEGMENT_FINISHED" state because
                         // "COPY_SEGMENT_STARTED" means copy didn't complete 
and we will count them later,
                         // "DELETE_SEGMENT_STARTED" means deletion failed in 
the previous attempt and we will retry later,
                         // "DELETE_SEGMENT_FINISHED" means deletion completed, 
so there is nothing to count.
@@ -1617,7 +1617,7 @@ public class RemoteLogManager implements Closeable, 
AsyncOffsetReader {
             log.updateHighestOffsetInRemoteStorage(offsetAndEpoch.offset());
         }
     }
-    
+
     private boolean deleteRemoteLogSegment(
         RemoteLogSegmentMetadata segmentMetadata,
         Predicate<RemoteLogSegmentMetadata> predicate
diff --git 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java
index c926b5a5409..d2bea0bd52b 100644
--- a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java
+++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java
@@ -27,9 +27,9 @@ import 
org.apache.kafka.common.record.internal.MutableRecordBatch;
 import org.apache.kafka.common.record.internal.Record;
 import org.apache.kafka.common.record.internal.RecordBatch;
 import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.common.utils.CloseableIterator;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
 import org.apache.kafka.storage.internals.utils.Throttler;
 
 import org.slf4j.Logger;
diff --git 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java
 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java
index dc4035b768e..591b778db2b 100644
--- 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java
+++ 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogValidator.java
@@ -34,10 +34,10 @@ import org.apache.kafka.common.record.internal.Record;
 import org.apache.kafka.common.record.internal.RecordBatch;
 import org.apache.kafka.common.requests.ProduceResponse.RecordError;
 import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.common.utils.CloseableIterator;
-import org.apache.kafka.common.utils.PrimitiveRef;
-import org.apache.kafka.common.utils.PrimitiveRef.LongRef;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.CloseableIterator;
+import org.apache.kafka.common.utils.internals.PrimitiveRef;
+import org.apache.kafka.common.utils.internals.PrimitiveRef.LongRef;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
diff --git 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java
index b358edd1392..b9f9ca3f8eb 100644
--- 
a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java
+++ 
b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java
@@ -43,9 +43,9 @@ import org.apache.kafka.common.record.internal.RecordVersion;
 import org.apache.kafka.common.record.internal.Records;
 import org.apache.kafka.common.requests.ListOffsetsRequest;
 import org.apache.kafka.common.utils.LogContext;
-import org.apache.kafka.common.utils.PrimitiveRef;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.internals.PrimitiveRef;
 import org.apache.kafka.server.common.OffsetAndEpoch;
 import org.apache.kafka.server.common.RequestLocal;
 import org.apache.kafka.server.common.TransactionVersion;
diff --git 
a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java
 
b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java
index 77fe1682cfb..d21ec04e0a4 100644
--- 
a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java
+++ 
b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogValidatorTest.java
@@ -34,8 +34,8 @@ import org.apache.kafka.common.record.internal.Record;
 import org.apache.kafka.common.record.internal.RecordBatch;
 import org.apache.kafka.common.record.internal.RecordVersion;
 import org.apache.kafka.common.record.internal.SimpleRecord;
-import org.apache.kafka.common.utils.PrimitiveRef;
 import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.internals.PrimitiveRef;
 import org.apache.kafka.server.common.RequestLocal;
 import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.storage.internals.log.LogValidator.ValidationResult;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
index 4f9ca84f4a2..12aa60c0c79 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
@@ -25,10 +25,10 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.apache.kafka.common.utils.FixedOrderMap;
 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.FixedOrderMap;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
 import org.apache.kafka.streams.errors.ErrorHandlerContext;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
index 7b3483f8e68..0992d33f700 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
@@ -18,8 +18,8 @@ package org.apache.kafka.streams.processor.internals;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.utils.FixedOrderMap;
 import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.internals.FixedOrderMap;
 import org.apache.kafka.streams.errors.ProcessorStateException;
 import org.apache.kafka.streams.errors.StreamsException;
 import org.apache.kafka.streams.errors.TaskCorruptedException;
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 70a19ff125f..7d25989083b 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,6 +17,7 @@
 package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 import org.apache.kafka.common.utils.internals.ByteUtils;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.ProcessorStateException;
@@ -260,7 +261,7 @@ class DualColumnFamilyAccessor extends 
AbstractColumnFamilyAccessor {
     }
 
     private static class RocksDBDualCFIterator
-        extends org.apache.kafka.common.utils.AbstractIterator<KeyValue<Bytes, 
byte[]>>
+        extends AbstractIterator<KeyValue<Bytes, byte[]>>
         implements ManagedKeyValueIterator<Bytes, byte[]> {
 
         // RocksDB's JNI interface does not expose getters/setters that allow 
the
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ListValueStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ListValueStore.java
index 83f01ca759d..358c2f0e1df 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ListValueStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ListValueStore.java
@@ -18,8 +18,8 @@ package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
-import org.apache.kafka.common.utils.AbstractIterator;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.state.KeyValueIterator;
 import org.apache.kafka.streams.state.KeyValueStore;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIterator.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIterator.java
index df26aa548ba..22e280add0e 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIterator.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbIterator.java
@@ -16,8 +16,8 @@
  */
 package org.apache.kafka.streams.state.internals;
 
-import org.apache.kafka.common.utils.AbstractIterator;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.common.utils.internals.AbstractIterator;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.InvalidStateStoreException;
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
index 4f5a837c00b..90bac45871a 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
@@ -17,8 +17,8 @@
 package org.apache.kafka.streams.state.internals;
 
 import org.apache.kafka.common.utils.Bytes;
-import org.apache.kafka.common.utils.CircularIterator;
 import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.internals.CircularIterator;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
 

Reply via email to