MINOR: Move MockTime to server-common (#13823)

This patch rewrite `MockTime` in Java and moves it to `server-common` module. This is a prerequisite to move `MockTimer` later on to `server-common` as well. 

Reviewers: David Arthur <mumrah@gmail.com>
diff --git a/build.gradle b/build.gradle
index fe1655b..b0b4a74 100644
--- a/build.gradle
+++ b/build.gradle
@@ -2541,6 +2541,7 @@
     implementation project(':connect:transforms')
     implementation project(':clients').sourceSets.test.output
     implementation project(':core').sourceSets.test.output
+    implementation project(':server-common').sourceSets.test.output
 
     implementation libs.jmhCore
     annotationProcessor libs.jmhGeneratorAnnProcess
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 73769b6..4de0671 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -338,6 +338,7 @@
     <allow pkg="org.apache.kafka.clients"/>
     <allow pkg="org.apache.kafka.clients.producer" exact-match="true"/>
     <allow pkg="org.apache.kafka.clients.consumer" exact-match="true"/>
+    <allow pkg="org.apache.kafka.server.util"/>
 
     <allow pkg="org.apache.kafka.streams"/>
 
diff --git a/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala b/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala
index da9cb63..9ca36a3 100644
--- a/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala
+++ b/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala
@@ -16,12 +16,12 @@
  */
 package kafka.common
 
-import kafka.utils.MockTime
 import org.apache.kafka.clients.{ClientRequest, ClientResponse, NetworkClient, RequestCompletionHandler}
 import org.apache.kafka.common.Node
 import org.apache.kafka.common.errors.{AuthenticationException, DisconnectException}
 import org.apache.kafka.common.protocol.ApiKeys
 import org.apache.kafka.common.requests.AbstractRequest
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.Test
 import org.mockito.ArgumentMatchers.{any, anyLong, same}
diff --git a/core/src/test/scala/kafka/metrics/LinuxIoMetricsCollectorTest.scala b/core/src/test/scala/kafka/metrics/LinuxIoMetricsCollectorTest.scala
index 5c208ed..db0487f 100644
--- a/core/src/test/scala/kafka/metrics/LinuxIoMetricsCollectorTest.scala
+++ b/core/src/test/scala/kafka/metrics/LinuxIoMetricsCollectorTest.scala
@@ -19,7 +19,8 @@
 
 import java.nio.charset.StandardCharsets
 import java.nio.file.Files
-import kafka.utils.{Logging, MockTime}
+import kafka.utils.Logging
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.test.TestUtils
 import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
 import org.junit.jupiter.api.{Test, Timeout}
diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
index 0f8f9eb..3b9c5f7 100644
--- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
+++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
@@ -19,7 +19,7 @@
 import kafka.log.UnifiedLog
 import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMillisProp, MetadataLogSegmentMinBytesProp, NodeIdProp, ProcessRolesProp, QuorumVotersProp}
 import kafka.server.{KafkaConfig, KafkaRaftServer}
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException}
 import org.apache.kafka.common.protocol
 import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable}
@@ -28,6 +28,7 @@
 import org.apache.kafka.raft._
 import org.apache.kafka.raft.internals.BatchBuilder
 import org.apache.kafka.server.common.serialization.RecordSerde
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
 import org.apache.kafka.storage.internals.log.{LogConfig, LogStartOffsetIncrementReason}
 import org.apache.kafka.test.TestUtils.assertOptional
diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
index 1c064fc..b5c19e2 100644
--- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
+++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
@@ -19,7 +19,7 @@
 
 import kafka.cluster.BrokerEndPoint
 import kafka.server.checkpoints.LazyOffsetCheckpoints
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.{Node, TopicPartition, Uuid}
 import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
 import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
@@ -30,7 +30,7 @@
 import org.apache.kafka.common.requests.LeaderAndIsrRequest
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.apache.kafka.server.common.OffsetAndEpoch
-import org.apache.kafka.server.util.MockScheduler
+import org.apache.kafka.server.util.{MockScheduler, MockTime}
 import org.apache.kafka.storage.internals.log.{AppendOrigin, LogDirFailureChannel}
 import org.junit.jupiter.api.{BeforeEach, Test}
 import org.junit.jupiter.api.Assertions._
diff --git a/core/src/test/scala/kafka/server/RemoteLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/RemoteLeaderEndPointTest.scala
index cac71c2..447ba34 100644
--- a/core/src/test/scala/kafka/server/RemoteLeaderEndPointTest.scala
+++ b/core/src/test/scala/kafka/server/RemoteLeaderEndPointTest.scala
@@ -21,7 +21,7 @@
 import kafka.log.UnifiedLog
 import kafka.server.AbstractFetcherThread.ResultWithPartitions
 import kafka.server.epoch.util.MockBlockingSender
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.clients.FetchSessionHandler
 import org.apache.kafka.common.errors.{FencedLeaderEpochException, UnknownLeaderEpochException}
 import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
@@ -32,6 +32,7 @@
 import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 import org.apache.kafka.server.common.{MetadataVersion, OffsetAndEpoch}
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{BeforeEach, Test}
 import org.junit.jupiter.params.ParameterizedTest
diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala
index 631a3af..9dda3fd 100755
--- a/core/src/test/scala/other/kafka/StressTestLog.scala
+++ b/core/src/test/scala/other/kafka/StressTestLog.scala
@@ -26,6 +26,7 @@
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record.FileRecords
 import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
 
 /**
diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala
index 4695e56..bf36059 100644
--- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala
@@ -21,7 +21,7 @@
 import kafka.server.checkpoints.OffsetCheckpoints
 import kafka.server.metadata.MockConfigRepository
 import kafka.utils.TestUtils.{MockAlterPartitionListener, MockAlterPartitionManager}
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
@@ -34,6 +34,7 @@
 import java.io.File
 import java.util.Properties
 import org.apache.kafka.server.common.MetadataVersion
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
 
 import scala.jdk.CollectionConverters._
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
index 11d2fcb..cbe417c 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
@@ -35,6 +35,7 @@
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.common.{TopicPartition, Uuid}
 import org.apache.kafka.server.common.MetadataVersion
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
 import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, FetchIsolation, FetchParams, LogAppendInfo, LogConfig, LogDirFailureChannel, ProducerStateManager, ProducerStateManagerConfig}
 import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index fce171c..afb4a52 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -53,7 +53,7 @@
 import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
-import org.apache.kafka.server.util.KafkaScheduler
+import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
 import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
 import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, EpochEntry, FetchIsolation, FetchParams, LogAppendInfo, LogDirFailureChannel, LogReadInfo, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig}
 import org.junit.jupiter.params.ParameterizedTest
diff --git a/core/src/test/scala/unit/kafka/cluster/ReplicaTest.scala b/core/src/test/scala/unit/kafka/cluster/ReplicaTest.scala
index 7012757..b4a4ee0 100644
--- a/core/src/test/scala/unit/kafka/cluster/ReplicaTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/ReplicaTest.scala
@@ -17,8 +17,8 @@
 package kafka.cluster
 
 import kafka.log.UnifiedLog
-import kafka.utils.MockTime
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.LogOffsetMetadata
 import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
 import org.junit.jupiter.api.{BeforeEach, Test}
diff --git a/core/src/test/scala/unit/kafka/controller/KafkaControllerTest.scala b/core/src/test/scala/unit/kafka/controller/KafkaControllerTest.scala
index 4ffe1fe..e9e8d3f 100644
--- a/core/src/test/scala/unit/kafka/controller/KafkaControllerTest.scala
+++ b/core/src/test/scala/unit/kafka/controller/KafkaControllerTest.scala
@@ -19,10 +19,11 @@
 
 import kafka.server.metadata.ZkMetadataCache
 import kafka.server.{BrokerFeatures, DelegationTokenManager, KafkaConfig}
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import kafka.zk.{BrokerInfo, KafkaZkClient}
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.server.metrics.KafkaMetricsGroup
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.{BeforeEach, Test}
 import org.mockito.ArgumentMatchers
 import org.mockito.ArgumentMatchers.{any, anyString}
diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala
index ab97b1c..23961be 100644
--- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala
@@ -31,7 +31,7 @@
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordConversionStats}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
-import org.apache.kafka.server.util.MockScheduler
+import org.apache.kafka.server.util.{MockScheduler, MockTime}
 import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig}
 import org.junit.jupiter.api.{AfterEach, BeforeEach}
 import org.mockito.Mockito.{CALLS_REAL_METHODS, mock, withSettings}
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala
index bcf24db..0a7b6f2 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorAdapterTest.scala
@@ -19,7 +19,6 @@
 import kafka.common.OffsetAndMetadata
 import kafka.coordinator.group.GroupCoordinatorConcurrencyTest.{JoinGroupCallback, SyncGroupCallback}
 import kafka.server.RequestLocal
-import kafka.utils.MockTime
 import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
 import org.apache.kafka.common.errors.{InvalidGroupIdException, UnsupportedVersionException}
 import org.apache.kafka.common.message.{ConsumerGroupHeartbeatRequestData, DeleteGroupsResponseData, DescribeGroupsResponseData, HeartbeatRequestData, HeartbeatResponseData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupRequestData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData}
@@ -33,6 +32,7 @@
 import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
 import org.apache.kafka.common.utils.{BufferSupplier, Time}
 import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.test.TestUtils.assertFutureThrows
 import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
 import org.junit.jupiter.api.Test
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
index 3a7f6f9..20d44fe 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
@@ -37,7 +37,7 @@
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
-import org.apache.kafka.server.util.KafkaScheduler
+import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
 import org.apache.kafka.storage.internals.log.AppendOrigin
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
index 03de648..479fc61 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
@@ -28,7 +28,7 @@
 import kafka.common.OffsetAndMetadata
 import kafka.log.UnifiedLog
 import kafka.server.{HostedPartition, KafkaConfig, ReplicaManager, RequestLocal}
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
 import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription
 import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
@@ -46,7 +46,7 @@
 import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.common.MetadataVersion._
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
-import org.apache.kafka.server.util.KafkaScheduler
+import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
 import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, FetchIsolation, LogAppendInfo, LogOffsetMetadata}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala
index 4c0f456..5d40c90 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala
@@ -16,10 +16,10 @@
  */
 package kafka.coordinator.transaction
 
-import kafka.utils.MockTime
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.record.RecordBatch
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.Test
 
diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
index 86f62bc..1575542 100644
--- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
@@ -20,12 +20,13 @@
 import java.nio.file.Files
 import java.util.Properties
 import kafka.server.BrokerTopicStats
-import kafka.utils.{MockTime, Pool, TestUtils}
+import kafka.utils.{Pool, TestUtils}
 import kafka.utils.Implicits._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch}
 import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
 import org.junit.jupiter.api.{AfterEach, Tag}
 
diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
index e8962b9..3d2cb30 100755
--- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
@@ -23,6 +23,7 @@
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.server.record.BrokerCompressionType
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api._
diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
index 444b4da..82fa5ff 100644
--- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
@@ -23,12 +23,12 @@
 import java.util.regex.Pattern
 import java.util.Collections
 import kafka.server.KafkaConfig
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.{KafkaException, TopicPartition}
 import org.apache.kafka.common.errors.KafkaStorageException
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
 import org.apache.kafka.common.utils.{Time, Utils}
-import org.apache.kafka.server.util.Scheduler
+import org.apache.kafka.server.util.{MockTime, Scheduler}
 import org.apache.kafka.storage.internals.log.{FetchDataInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
index efdcae2..925ab79 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -18,12 +18,12 @@
 package kafka.log
 
 import java.io.PrintWriter
-
 import com.yammer.metrics.core.{Gauge, MetricName}
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.record.{CompressionType, RecordBatch}
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, Test}
 
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
index 9256f30..b2cc219 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
@@ -20,6 +20,7 @@
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.record.CompressionType
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.{Arguments, MethodSource}
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
index cc52b32..f11b315 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
@@ -26,6 +26,7 @@
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, LogDirFailureChannel, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, Test}
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
index 9dc9408..5ab5369 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
@@ -26,6 +26,7 @@
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record._
 import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_11_0_IV0, IBP_0_9_0}
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.extension.ExtensionContext
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
index 79054b6..5a3ce6e 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -26,6 +26,7 @@
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.server.metrics.KafkaMetricsGroup
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanerConfig, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogStartOffsetIncrementReason, OffsetMap, ProducerStateManager, ProducerStateManagerConfig}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, Test}
diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
index f0e2773..64527c7 100644
--- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
@@ -23,7 +23,7 @@
 import java.util.{Optional, OptionalLong, Properties}
 import kafka.server.{BrokerTopicStats, KafkaConfig}
 import kafka.server.metadata.MockConfigRepository
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.errors.KafkaStorageException
@@ -31,7 +31,7 @@
 import org.apache.kafka.common.utils.{Time, Utils}
 import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0
-import org.apache.kafka.server.util.Scheduler
+import org.apache.kafka.server.util.{MockTime, Scheduler}
 import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
 import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, EpochEntry, FetchDataInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile}
 import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index 68f6cbb..dff07ce 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -39,6 +39,7 @@
 import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future}
 import java.util.{Collections, Properties}
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
 
 import scala.collection.{Map, mutable}
diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
index daea156..4f54e21 100644
--- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
@@ -19,7 +19,6 @@
 import java.nio.ByteBuffer
 import java.util.concurrent.TimeUnit
 import kafka.server.{BrokerTopicStats, RequestLocal}
-import kafka.utils.MockTime
 import kafka.utils.TestUtils.meterCount
 import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException}
 import org.apache.kafka.common.record._
@@ -28,6 +27,7 @@
 import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.storage.internals.log.LogValidator.ValidationResult
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{AppendOrigin, LogValidator, RecordValidationException}
 import org.apache.kafka.test.TestUtils
 import org.junit.jupiter.api.Assertions._
diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
index c52320c..0444f94 100755
--- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
@@ -33,7 +33,7 @@
 import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils}
 import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
-import org.apache.kafka.server.util.{KafkaScheduler, Scheduler}
+import org.apache.kafka.server.util.{KafkaScheduler, MockTime, Scheduler}
 import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile
 import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
 import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, EpochEntry, FetchIsolation, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig, RecordValidationException}
diff --git a/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala b/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala
index 6e2edcb..98e3ab8 100644
--- a/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala
+++ b/core/src/test/scala/unit/kafka/log/remote/RemoteIndexCacheTest.scala
@@ -17,10 +17,10 @@
 package kafka.log.remote
 
 import kafka.log.UnifiedLog
-import kafka.utils.MockTime
 import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
 import org.apache.kafka.server.log.remote.storage.RemoteStorageManager.IndexType
 import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteStorageManager}
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{OffsetIndex, OffsetPosition, TimeIndex}
 import org.apache.kafka.test.TestUtils
 import org.junit.jupiter.api.Assertions._
diff --git a/core/src/test/scala/unit/kafka/network/ConnectionQuotasTest.scala b/core/src/test/scala/unit/kafka/network/ConnectionQuotasTest.scala
index 725435c..59be8a3 100644
--- a/core/src/test/scala/unit/kafka/network/ConnectionQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/network/ConnectionQuotasTest.scala
@@ -25,7 +25,7 @@
 import kafka.network.Processor.ListenerMetricTag
 import kafka.server.KafkaConfig
 import kafka.utils.Implicits.MapExtensionMethods
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.config.ConfigException
 import org.apache.kafka.common.config.internals.QuotaConfigs
 import org.apache.kafka.common.metrics.internals.MetricsUtils
@@ -33,6 +33,7 @@
 import org.apache.kafka.common.network._
 import org.apache.kafka.common.utils.Time
 import org.apache.kafka.server.metrics.KafkaMetricsGroup
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api._
 
diff --git a/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala b/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala
index 10e7bcc..880d484 100644
--- a/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala
@@ -20,7 +20,6 @@
 import java.util.Collections
 import java.util.stream.{Stream => JStream}
 import kafka.api.LeaderAndIsr
-import kafka.utils.MockTime
 import kafka.zk.KafkaZkClient
 import org.apache.kafka.clients.ClientResponse
 import org.apache.kafka.common.TopicIdPartition
@@ -36,7 +35,7 @@
 import org.apache.kafka.metadata.LeaderRecoveryState
 import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV2, IBP_3_2_IV0, IBP_3_5_IV1}
-import org.apache.kafka.server.util.MockScheduler
+import org.apache.kafka.server.util.{MockScheduler, MockTime}
 import org.apache.kafka.test.TestUtils.assertFutureThrows
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.BeforeEach
diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
index 9d93cb8..1977a29 100644
--- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala
@@ -19,8 +19,7 @@
 
 import java.util.{Collections, Properties}
 import java.util.concurrent.atomic.{AtomicLong, AtomicReference}
-
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.clients.{Metadata, MockClient, NodeApiVersions}
 import org.apache.kafka.common.config.SaslConfigs
 import org.apache.kafka.common.Node
@@ -35,6 +34,7 @@
 import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.apache.kafka.common.utils.LogContext
 import org.apache.kafka.metadata.BrokerState
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.{Test, Timeout}
 import org.junit.jupiter.api.Assertions._
 
diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
index e55c8fb..5243a21 100644
--- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
@@ -21,7 +21,7 @@
 import kafka.raft.RaftManager
 import kafka.server.QuotaFactory.QuotaManagers
 import kafka.test.MockController
-import kafka.utils.{MockTime, NotNothing}
+import kafka.utils.NotNothing
 import org.apache.kafka.clients.admin.AlterConfigOp
 import org.apache.kafka.common.Uuid.ZERO_UUID
 import org.apache.kafka.common.acl.AclOperation
@@ -51,6 +51,7 @@
 import org.apache.kafka.controller.{Controller, ControllerRequestContext, ResultOrError}
 import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer}
 import org.apache.kafka.server.common.{ApiMessageAndVersion, ProducerIdsBlock}
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, Test}
 import org.junit.jupiter.params.ParameterizedTest
diff --git a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
index 2275673..487e5c5 100755
--- a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
@@ -16,7 +16,6 @@
 */
 package kafka.server
 
-import kafka.utils.MockTime
 import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
 import org.apache.kafka.common.message.FetchResponseData
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
@@ -26,6 +25,7 @@
 import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID}
 import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata}
 import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{Test, Timeout}
 import org.junit.jupiter.params.ParameterizedTest
@@ -34,7 +34,6 @@
 import scala.jdk.CollectionConverters._
 import java.util
 import java.util.{Collections, Optional}
-
 import scala.collection.mutable.ArrayBuffer
 
 @Timeout(120)
diff --git a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
index 21c7d0d..b2a1906 100644
--- a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
@@ -20,10 +20,8 @@
 import java.nio.ByteBuffer
 import java.util.Optional
 import java.util.concurrent.atomic.AtomicReference
-
 import kafka.network
 import kafka.network.RequestChannel
-import kafka.utils.MockTime
 import org.apache.kafka.clients.{MockClient, NodeApiVersions}
 import org.apache.kafka.clients.MockClient.RequestMatcher
 import org.apache.kafka.common.Node
@@ -36,6 +34,7 @@
 import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AlterConfigsRequest, AlterConfigsResponse, EnvelopeRequest, EnvelopeResponse, RequestContext, RequestHeader, RequestTestUtils}
 import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
 import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.Test
 import org.mockito.Mockito
diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
index 48954c7..170e2e4 100755
--- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
@@ -22,12 +22,12 @@
 import org.apache.kafka.metadata.LeaderRecoveryState
 import org.junit.jupiter.api._
 import org.junit.jupiter.api.Assertions._
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import kafka.cluster.Partition
 import kafka.server.metadata.MockConfigRepository
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.record.SimpleRecord
-import org.apache.kafka.server.util.KafkaScheduler
+import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
 import org.apache.kafka.storage.internals.log.{CleanerConfig, LogDirFailureChannel}
 
 class HighwatermarkPersistenceTest {
diff --git a/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala b/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala
index 4f401f2..ecd29c3 100644
--- a/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala
+++ b/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala
@@ -18,7 +18,6 @@
 
 import java.io.File
 import java.util.Properties
-
 import kafka.cluster.Partition
 import kafka.log.{LogManager, UnifiedLog}
 import kafka.server.QuotaFactory.QuotaManagers
@@ -28,6 +27,7 @@
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.utils.Time
 import org.apache.kafka.metadata.LeaderRecoveryState
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{LogDirFailureChannel, LogOffsetMetadata}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index 3c16f4d..40023dd 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -30,7 +30,7 @@
 import kafka.network.RequestChannel
 import kafka.server.QuotaFactory.QuotaManagers
 import kafka.server.metadata.{ConfigRepository, KRaftMetadataCache, MockConfigRepository, ZkMetadataCache}
-import kafka.utils.{Log4jController, MockTime, TestUtils}
+import kafka.utils.{Log4jController, TestUtils}
 import kafka.zk.KafkaZkClient
 import org.apache.kafka.clients.admin.AlterConfigOp.OpType
 import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry}
@@ -95,6 +95,7 @@
 import org.apache.kafka.coordinator.group.GroupCoordinator
 import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1}
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchParams, FetchPartitionData}
 
 class KafkaApisTest {
diff --git a/core/src/test/scala/unit/kafka/server/MockBrokerToControllerChannelManager.scala b/core/src/test/scala/unit/kafka/server/MockBrokerToControllerChannelManager.scala
index 1752e2c..4626c47 100644
--- a/core/src/test/scala/unit/kafka/server/MockBrokerToControllerChannelManager.scala
+++ b/core/src/test/scala/unit/kafka/server/MockBrokerToControllerChannelManager.scala
@@ -16,10 +16,10 @@
  */
 package kafka.server
 
-import kafka.utils.MockTime
 import org.apache.kafka.clients.{ClientResponse, MockClient, NodeApiVersions}
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.AbstractRequest
+import org.apache.kafka.server.util.MockTime
 
 class MockBrokerToControllerChannelManager(
   val client: MockClient,
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
index ad07b70..00d7d61 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerConcurrencyTest.scala
@@ -24,7 +24,7 @@
 import kafka.server.metadata.KRaftMetadataCache
 import kafka.server.metadata.MockConfigRepository
 import kafka.utils.TestUtils.waitUntilTrue
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.metadata.RegisterBrokerRecord
 import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord, TopicRecord}
 import org.apache.kafka.common.metrics.Metrics
@@ -38,7 +38,7 @@
 import org.apache.kafka.image.{MetadataDelta, MetadataImage}
 import org.apache.kafka.metadata.LeaderRecoveryState
 import org.apache.kafka.metadata.PartitionRegistration
-import org.apache.kafka.server.util.ShutdownableThread
+import org.apache.kafka.server.util.{MockTime, ShutdownableThread}
 import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData, LogConfig, LogDirFailureChannel}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, Test}
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
index 4e35084..320d766 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -29,7 +29,7 @@
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
 import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
 import org.apache.kafka.metadata.LeaderRecoveryState
-import org.apache.kafka.server.util.KafkaScheduler
+import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
 import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, FetchParams, LogDirFailureChannel, LogOffsetMetadata, LogOffsetSnapshot}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, Test}
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index e5a8e18..ca93339 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -33,7 +33,7 @@
 import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile}
 import kafka.server.epoch.util.MockBlockingSender
 import kafka.utils.timer.MockTimer
-import kafka.utils.{MockTime, Pool, TestUtils}
+import kafka.utils.{Pool, TestUtils}
 import org.apache.kafka.clients.FetchSessionHandler
 import org.apache.kafka.common.errors.{InvalidPidMappingException, KafkaStorageException}
 import org.apache.kafka.common.message.LeaderAndIsrRequestData
@@ -60,7 +60,7 @@
 import org.apache.kafka.server.common.OffsetAndEpoch
 import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
 import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
-import org.apache.kafka.server.util.MockScheduler
+import org.apache.kafka.server.util.{MockScheduler, MockTime}
 import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
index ad5a7ce..434ce24 100644
--- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
+++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
@@ -17,11 +17,10 @@
 package kafka.server.epoch
 
 import java.io.File
-
-import kafka.log.{UnifiedLog, LogManager}
+import kafka.log.{LogManager, UnifiedLog}
 import kafka.server.QuotaFactory.QuotaManagers
 import kafka.server._
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic}
 import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult}
@@ -30,6 +29,7 @@
 import org.apache.kafka.common.record.RecordBatch
 import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
 import org.apache.kafka.server.common.OffsetAndEpoch
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.LogDirFailureChannel
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala
index 9ad65a3..29324fe 100644
--- a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala
@@ -22,7 +22,7 @@
 import java.util.{HashMap, Optional, Map => JMap}
 import java.time.Duration
 import kafka.tools.ConsoleConsumer.ConsumerWrapper
-import kafka.utils.{Exit, MockTime, TestUtils}
+import kafka.utils.{Exit, TestUtils}
 import org.apache.kafka.clients.consumer.{ConsumerRecord, MockConsumer, OffsetResetStrategy}
 import org.apache.kafka.common.{MessageFormatter, TopicPartition}
 import org.apache.kafka.common.record.TimestampType
@@ -35,6 +35,7 @@
 import org.apache.kafka.clients.consumer.ConsumerRecords
 import org.apache.kafka.common.errors.TimeoutException
 import org.apache.kafka.common.header.internals.RecordHeaders
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{BeforeEach, Test}
 
diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
index e28694b..7cb2627 100644
--- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
@@ -25,7 +25,7 @@
 import kafka.raft.{KafkaMetadataLog, MetadataLogConfig}
 import kafka.server.{BrokerTopicStats, KafkaRaftServer}
 import kafka.tools.DumpLogSegments.TimeIndexDumpErrors
-import kafka.utils.{MockTime, TestUtils}
+import kafka.utils.TestUtils
 import org.apache.kafka.common.Uuid
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.memory.MemoryPool
@@ -36,6 +36,7 @@
 import org.apache.kafka.metadata.MetadataRecordSerde
 import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch}
 import org.apache.kafka.server.common.ApiMessageAndVersion
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.snapshot.RecordsSnapshotWriter
 import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
 import org.junit.jupiter.api.Assertions._
diff --git a/core/src/test/scala/unit/kafka/utils/QuotaUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/QuotaUtilsTest.scala
index 949618d..0c0fd3a 100755
--- a/core/src/test/scala/unit/kafka/utils/QuotaUtilsTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/QuotaUtilsTest.scala
@@ -18,10 +18,10 @@
 package kafka.utils
 
 import java.util.concurrent.TimeUnit
-
 import org.apache.kafka.common.MetricName
 import org.apache.kafka.common.metrics.{KafkaMetric, MetricConfig, Quota, QuotaViolationException}
 import org.apache.kafka.common.metrics.stats.{Rate, Value}
+import org.apache.kafka.server.util.MockTime
 
 import scala.jdk.CollectionConverters._
 import org.junit.jupiter.api.Assertions._
diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
index a8cd741..1aa04d6 100644
--- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
@@ -22,7 +22,7 @@
 import kafka.log.{LocalLog, LogLoader, LogSegments, UnifiedLog}
 import kafka.server.BrokerTopicStats
 import kafka.utils.TestUtils.retry
-import org.apache.kafka.server.util.KafkaScheduler
+import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
 import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManager, ProducerStateManagerConfig}
 import org.junit.jupiter.api.Assertions._
 import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, Timeout}
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 0a597fd..9616f89 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -71,6 +71,7 @@
 import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer}
 import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
+import org.apache.kafka.server.util.MockTime
 import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
 import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
 import org.apache.zookeeper.KeeperException.SessionExpiredException
diff --git a/core/src/test/scala/unit/kafka/utils/ThrottlerTest.scala b/core/src/test/scala/unit/kafka/utils/ThrottlerTest.scala
index 80ebde4..896eac2 100755
--- a/core/src/test/scala/unit/kafka/utils/ThrottlerTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/ThrottlerTest.scala
@@ -17,6 +17,7 @@
 
 package kafka.utils
 
+import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Test
 import org.junit.jupiter.api.Assertions.{assertTrue, assertEquals}
 
diff --git a/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala b/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala
index 819954a..f0e1cb9 100644
--- a/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala
+++ b/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala
@@ -16,7 +16,7 @@
   */
 package kafka.utils.timer
 
-import kafka.utils.MockTime
+import org.apache.kafka.server.util.MockTime
 
 import scala.collection.mutable
 
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java
index 59afc76..12e2cd1 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java
@@ -21,6 +21,7 @@
 import kafka.cluster.DelayedOperations;
 import kafka.cluster.AlterPartitionListener;
 import kafka.cluster.Partition;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.storage.internals.log.LogAppendInfo;
 import kafka.log.LogManager;
 import kafka.server.AlterPartitionManager;
@@ -42,7 +43,6 @@
 import kafka.server.checkpoints.OffsetCheckpoints;
 import kafka.server.metadata.MockConfigRepository;
 import kafka.server.metadata.ZkMetadataCache;
-import kafka.utils.MockTime;
 import kafka.utils.Pool;
 import kafka.utils.TestUtils;
 import kafka.zk.KafkaZkClient;
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
index ff3e04a..7f3bc14 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
@@ -22,6 +22,7 @@
 import kafka.server.BrokerFeatures;
 import kafka.server.BrokerTopicStats;
 import kafka.server.KafkaConfig;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.storage.internals.log.CleanerConfig;
 import org.apache.kafka.storage.internals.log.LogConfig;
 import org.apache.kafka.storage.internals.log.LogDirFailureChannel;
@@ -31,7 +32,6 @@
 import kafka.server.builders.ReplicaManagerBuilder;
 import kafka.server.checkpoints.OffsetCheckpoints;
 import kafka.server.metadata.MockConfigRepository;
-import kafka.utils.MockTime;
 import kafka.utils.TestUtils;
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.TopicPartition;
diff --git a/core/src/test/scala/unit/kafka/utils/MockTime.scala b/server-common/src/test/java/org/apache/kafka/server/util/MockTime.java
similarity index 61%
rename from core/src/test/scala/unit/kafka/utils/MockTime.scala
rename to server-common/src/test/java/org/apache/kafka/server/util/MockTime.java
index 2c7c221..6d18d08 100644
--- a/core/src/test/scala/unit/kafka/utils/MockTime.scala
+++ b/server-common/src/test/java/org/apache/kafka/server/util/MockTime.java
@@ -1,11 +1,11 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
- * 
+ * 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
@@ -14,11 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package kafka.utils
-
-import org.apache.kafka.common.utils.{MockTime => JMockTime}
-import org.apache.kafka.server.util.MockScheduler
+package org.apache.kafka.server.util;
 
 /**
  * A class used for unit testing things which depend on the Time interface.
@@ -27,15 +23,21 @@
  * 1. This has an associated scheduler instance for managing background tasks in a deterministic way.
  * 2. This doesn't support the `auto-tick` functionality as it interacts badly with the current implementation of `MockScheduler`.
  */
-class MockTime(currentTimeMs: Long, currentHiResTimeNs: Long) extends JMockTime(0, currentTimeMs, currentHiResTimeNs) {
+public class MockTime extends org.apache.kafka.common.utils.MockTime {
+    public final MockScheduler scheduler;
 
-  def this() = this(System.currentTimeMillis(), System.nanoTime())
+    public MockTime() {
+        this(System.currentTimeMillis(), System.nanoTime());
+    }
 
-  val scheduler = new MockScheduler(this)
+    public MockTime(long currentTimeMs, long currentHiResTimeNs) {
+        super(0L, currentTimeMs, currentHiResTimeNs);
+        scheduler = new MockScheduler(this);
+    }
 
-  override def sleep(ms: Long): Unit = {
-    super.sleep(ms)
-    scheduler.tick()
-  }
-
+    @Override
+    public void sleep(long ms) {
+        super.sleep(ms);
+        scheduler.tick();
+    }
 }
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java
index d05a578..732642d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java
@@ -17,7 +17,6 @@
 package org.apache.kafka.streams.integration;
 
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
@@ -26,6 +25,7 @@
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java
index b2272d0..c6ef6c7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java
@@ -19,7 +19,6 @@
 import java.io.File;
 import java.util.Collections;
 import java.util.concurrent.atomic.AtomicReference;
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.TopicPartition;
@@ -27,6 +26,7 @@
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
index b4b20d2..e184b4d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
@@ -16,12 +16,12 @@
  */
 package org.apache.kafka.streams.integration;
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.LongSerializer;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KafkaStreams.State;
 import org.apache.kafka.streams.KeyValue;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
index 587c303..274cdb0 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.integration;
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.admin.Admin;
 import org.apache.kafka.clients.admin.AdminClientConfig;
 import org.apache.kafka.clients.admin.Config;
@@ -28,6 +27,7 @@
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.StreamsBuilder;
 import org.apache.kafka.streams.StreamsConfig;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
index 9809171..cf1732b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.integration;
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.IntegerSerializer;
@@ -24,6 +23,7 @@
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.KeyValueTimestamp;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
index 9a66a7d..27f0929 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
@@ -17,7 +17,6 @@
 package org.apache.kafka.streams.integration;
 
 import kafka.tools.ConsoleConsumer;
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.IntegerDeserializer;
@@ -27,6 +26,7 @@
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
index 7e60441..cc55e57 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKStreamIntegrationTest.java
@@ -17,12 +17,12 @@
 
 package org.apache.kafka.streams.integration;
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest.java
index ad2901f..a065cf1 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest.java
@@ -31,6 +31,7 @@
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
@@ -51,7 +52,6 @@
 import org.apache.kafka.streams.utils.UniqueTopicSerdeScope;
 import org.apache.kafka.test.TestUtils;
 
-import kafka.utils.MockTime;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeAll;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java
index 08f4fe5..997d846 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.integration;
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.serialization.FloatSerializer;
@@ -27,6 +26,7 @@
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java
index ce21132..9fe78fb 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.integration;
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.serialization.LongDeserializer;
@@ -24,6 +23,7 @@
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KafkaStreamsWrapper;
 import org.apache.kafka.streams.KeyValue;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
index 92d6321..f3c9bcc 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.integration;
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerConfig;
@@ -28,6 +27,7 @@
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
 import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KafkaStreams.State;
 import org.apache.kafka.streams.KafkaStreamsTest;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index ccebed8..bfbf428 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.integration;
 
-import kafka.utils.MockTime;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
@@ -27,6 +26,7 @@
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.streams.KafkaStreams;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
index d990530..f5dd9fe 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java
@@ -19,11 +19,11 @@
 import kafka.server.ConfigType;
 import kafka.server.KafkaConfig;
 import kafka.server.KafkaServer;
-import kafka.utils.MockTime;
 import kafka.zk.EmbeddedZookeeper;
 import org.apache.kafka.clients.admin.Admin;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+import org.apache.kafka.server.util.MockTime;
 import org.apache.kafka.test.TestCondition;
 import org.apache.kafka.test.TestUtils;
 import org.slf4j.Logger;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
index 165aaed..8936938 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java
@@ -19,7 +19,6 @@
 import kafka.cluster.EndPoint;
 import kafka.server.KafkaConfig;
 import kafka.server.KafkaServer;
-import kafka.utils.MockTime;
 import kafka.utils.TestUtils;
 import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.admin.Admin;
@@ -29,6 +28,7 @@
 import org.apache.kafka.common.config.types.Password;
 import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
 import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.util.MockTime;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;