mirror of https://github.com/apache/kafka.git
MINOR: remove the copy constructor of LogSegment (#15488)
In the LogSegment, the copy constructor is only used in LogLoaderTest Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
432825df95
commit
3fcaa9ccc0
|
|
@ -34,14 +34,14 @@ import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0
|
|||
import org.apache.kafka.server.config.Defaults
|
||||
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, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile}
|
||||
import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, EpochEntry, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile}
|
||||
import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler
|
||||
import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
|
||||
import org.junit.jupiter.api.function.Executable
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.CsvSource
|
||||
import org.mockito.ArgumentMatchers
|
||||
import org.mockito.{ArgumentMatchers, Mockito}
|
||||
import org.mockito.ArgumentMatchers.{any, anyLong}
|
||||
import org.mockito.Mockito.{mock, reset, times, verify, when}
|
||||
|
||||
|
|
@ -352,19 +352,15 @@ class LogLoaderTest {
|
|||
// Intercept all segment read calls
|
||||
val interceptedLogSegments = new LogSegments(topicPartition) {
|
||||
override def add(segment: LogSegment): LogSegment = {
|
||||
val wrapper = new LogSegment(segment) {
|
||||
|
||||
override def read(startOffset: Long, maxSize: Int, maxPosition: Long, minOneMessage: Boolean): FetchDataInfo = {
|
||||
segmentsWithReads += this
|
||||
super.read(startOffset, maxSize, maxPosition, minOneMessage)
|
||||
}
|
||||
|
||||
override def recover(producerStateManager: ProducerStateManager,
|
||||
leaderEpochCache: Optional[LeaderEpochFileCache]): Int = {
|
||||
recoveredSegments += this
|
||||
super.recover(producerStateManager, leaderEpochCache)
|
||||
}
|
||||
}
|
||||
val wrapper = Mockito.spy(segment)
|
||||
Mockito.doAnswer { in =>
|
||||
segmentsWithReads += wrapper
|
||||
segment.read(in.getArgument(0, classOf[java.lang.Long]), in.getArgument(1, classOf[java.lang.Integer]), in.getArgument(2, classOf[java.lang.Long]), in.getArgument(3, classOf[java.lang.Boolean]))
|
||||
}.when(wrapper).read(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())
|
||||
Mockito.doAnswer { in =>
|
||||
recoveredSegments += wrapper
|
||||
segment.recover(in.getArgument(0, classOf[ProducerStateManager]), in.getArgument(1, classOf[Optional[LeaderEpochFileCache]]))
|
||||
}.when(wrapper).recover(ArgumentMatchers.any(), ArgumentMatchers.any())
|
||||
super.add(wrapper)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -128,12 +128,6 @@ public class LogSegment implements Closeable {
|
|||
this.created = time.milliseconds();
|
||||
}
|
||||
|
||||
// Visible for testing
|
||||
public LogSegment(LogSegment segment) {
|
||||
this(segment.log, segment.lazyOffsetIndex, segment.lazyTimeIndex, segment.txnIndex, segment.baseOffset,
|
||||
segment.indexIntervalBytes, segment.rollJitterMs, segment.time);
|
||||
}
|
||||
|
||||
public OffsetIndex offsetIndex() throws IOException {
|
||||
return lazyOffsetIndex.get();
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue