mirror of https://github.com/apache/kafka.git
KAFKA-18479: Remove keepPartitionMetadataFile in UnifiedLog and LogMan… (#18491)
Reviewers: Jun Rao <junrao@gmail.com>
This commit is contained in:
parent
d82f03e98b
commit
92fd99bda1
|
@ -56,7 +56,6 @@ public class LogManagerBuilder {
|
||||||
private BrokerTopicStats brokerTopicStats = null;
|
private BrokerTopicStats brokerTopicStats = null;
|
||||||
private LogDirFailureChannel logDirFailureChannel = null;
|
private LogDirFailureChannel logDirFailureChannel = null;
|
||||||
private Time time = Time.SYSTEM;
|
private Time time = Time.SYSTEM;
|
||||||
private boolean keepPartitionMetadataFile = true;
|
|
||||||
private boolean remoteStorageSystemEnable = false;
|
private boolean remoteStorageSystemEnable = false;
|
||||||
private long initialTaskDelayMs = ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT;
|
private long initialTaskDelayMs = ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT;
|
||||||
|
|
||||||
|
@ -145,11 +144,6 @@ public class LogManagerBuilder {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public LogManagerBuilder setKeepPartitionMetadataFile(boolean keepPartitionMetadataFile) {
|
|
||||||
this.keepPartitionMetadataFile = keepPartitionMetadataFile;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public LogManagerBuilder setRemoteStorageSystemEnable(boolean remoteStorageSystemEnable) {
|
public LogManagerBuilder setRemoteStorageSystemEnable(boolean remoteStorageSystemEnable) {
|
||||||
this.remoteStorageSystemEnable = remoteStorageSystemEnable;
|
this.remoteStorageSystemEnable = remoteStorageSystemEnable;
|
||||||
return this;
|
return this;
|
||||||
|
@ -186,7 +180,6 @@ public class LogManagerBuilder {
|
||||||
brokerTopicStats,
|
brokerTopicStats,
|
||||||
logDirFailureChannel,
|
logDirFailureChannel,
|
||||||
time,
|
time,
|
||||||
keepPartitionMetadataFile,
|
|
||||||
remoteStorageSystemEnable,
|
remoteStorageSystemEnable,
|
||||||
initialTaskDelayMs);
|
initialTaskDelayMs);
|
||||||
}
|
}
|
||||||
|
|
|
@ -78,7 +78,6 @@ class LogManager(logDirs: Seq[File],
|
||||||
brokerTopicStats: BrokerTopicStats,
|
brokerTopicStats: BrokerTopicStats,
|
||||||
logDirFailureChannel: LogDirFailureChannel,
|
logDirFailureChannel: LogDirFailureChannel,
|
||||||
time: Time,
|
time: Time,
|
||||||
val keepPartitionMetadataFile: Boolean,
|
|
||||||
remoteStorageSystemEnable: Boolean,
|
remoteStorageSystemEnable: Boolean,
|
||||||
val initialTaskDelayMs: Long) extends Logging {
|
val initialTaskDelayMs: Long) extends Logging {
|
||||||
|
|
||||||
|
@ -346,7 +345,6 @@ class LogManager(logDirs: Seq[File],
|
||||||
logDirFailureChannel = logDirFailureChannel,
|
logDirFailureChannel = logDirFailureChannel,
|
||||||
lastShutdownClean = hadCleanShutdown,
|
lastShutdownClean = hadCleanShutdown,
|
||||||
topicId = None,
|
topicId = None,
|
||||||
keepPartitionMetadataFile = keepPartitionMetadataFile,
|
|
||||||
numRemainingSegments = numRemainingSegments,
|
numRemainingSegments = numRemainingSegments,
|
||||||
remoteStorageSystemEnable = remoteStorageSystemEnable)
|
remoteStorageSystemEnable = remoteStorageSystemEnable)
|
||||||
|
|
||||||
|
@ -1074,7 +1072,6 @@ class LogManager(logDirs: Seq[File],
|
||||||
brokerTopicStats = brokerTopicStats,
|
brokerTopicStats = brokerTopicStats,
|
||||||
logDirFailureChannel = logDirFailureChannel,
|
logDirFailureChannel = logDirFailureChannel,
|
||||||
topicId = topicId,
|
topicId = topicId,
|
||||||
keepPartitionMetadataFile = keepPartitionMetadataFile,
|
|
||||||
remoteStorageSystemEnable = remoteStorageSystemEnable)
|
remoteStorageSystemEnable = remoteStorageSystemEnable)
|
||||||
|
|
||||||
if (isFuture)
|
if (isFuture)
|
||||||
|
@ -1552,8 +1549,7 @@ object LogManager {
|
||||||
kafkaScheduler: Scheduler,
|
kafkaScheduler: Scheduler,
|
||||||
time: Time,
|
time: Time,
|
||||||
brokerTopicStats: BrokerTopicStats,
|
brokerTopicStats: BrokerTopicStats,
|
||||||
logDirFailureChannel: LogDirFailureChannel,
|
logDirFailureChannel: LogDirFailureChannel): LogManager = {
|
||||||
keepPartitionMetadataFile: Boolean): LogManager = {
|
|
||||||
val defaultProps = config.extractLogConfigMap
|
val defaultProps = config.extractLogConfigMap
|
||||||
|
|
||||||
LogConfig.validateBrokerLogConfigValues(defaultProps, config.remoteLogManagerConfig.isRemoteStorageSystemEnabled())
|
LogConfig.validateBrokerLogConfigValues(defaultProps, config.remoteLogManagerConfig.isRemoteStorageSystemEnabled())
|
||||||
|
@ -1578,7 +1574,6 @@ object LogManager {
|
||||||
brokerTopicStats = brokerTopicStats,
|
brokerTopicStats = brokerTopicStats,
|
||||||
logDirFailureChannel = logDirFailureChannel,
|
logDirFailureChannel = logDirFailureChannel,
|
||||||
time = time,
|
time = time,
|
||||||
keepPartitionMetadataFile = keepPartitionMetadataFile,
|
|
||||||
interBrokerProtocolVersion = config.interBrokerProtocolVersion,
|
interBrokerProtocolVersion = config.interBrokerProtocolVersion,
|
||||||
remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled(),
|
remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled(),
|
||||||
initialTaskDelayMs = config.logInitialTaskDelayMs)
|
initialTaskDelayMs = config.logInitialTaskDelayMs)
|
||||||
|
|
|
@ -85,13 +85,6 @@ import scala.jdk.OptionConverters.{RichOption, RichOptional, RichOptionalInt}
|
||||||
* @param _topicId optional Uuid to specify the topic ID for the topic if it exists. Should only be specified when
|
* @param _topicId optional Uuid to specify the topic ID for the topic if it exists. Should only be specified when
|
||||||
* first creating the log through Partition.makeLeader or Partition.makeFollower. When reloading a log,
|
* first creating the log through Partition.makeLeader or Partition.makeFollower. When reloading a log,
|
||||||
* this field will be populated by reading the topic ID value from partition.metadata if it exists.
|
* this field will be populated by reading the topic ID value from partition.metadata if it exists.
|
||||||
* @param keepPartitionMetadataFile boolean flag to indicate whether the partition.metadata file should be kept in the
|
|
||||||
* log directory. A partition.metadata file is only created when the raft controller is used
|
|
||||||
* or the ZK controller and this broker's inter-broker protocol version is at least 2.8.
|
|
||||||
* This file will persist the topic ID on the broker. If inter-broker protocol for a ZK controller
|
|
||||||
* is downgraded below 2.8, a topic ID may be lost and a new ID generated upon re-upgrade.
|
|
||||||
* If the inter-broker protocol version on a ZK cluster is below 2.8, partition.metadata
|
|
||||||
* will be deleted to avoid ID conflicts upon re-upgrade.
|
|
||||||
* @param remoteStorageSystemEnable flag to indicate whether the system level remote log storage is enabled or not.
|
* @param remoteStorageSystemEnable flag to indicate whether the system level remote log storage is enabled or not.
|
||||||
*/
|
*/
|
||||||
@threadsafe
|
@threadsafe
|
||||||
|
@ -102,7 +95,6 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
@volatile var leaderEpochCache: LeaderEpochFileCache,
|
@volatile var leaderEpochCache: LeaderEpochFileCache,
|
||||||
val producerStateManager: ProducerStateManager,
|
val producerStateManager: ProducerStateManager,
|
||||||
@volatile private var _topicId: Option[Uuid],
|
@volatile private var _topicId: Option[Uuid],
|
||||||
val keepPartitionMetadataFile: Boolean,
|
|
||||||
val remoteStorageSystemEnable: Boolean = false,
|
val remoteStorageSystemEnable: Boolean = false,
|
||||||
@volatile private var logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER) extends Logging with AutoCloseable {
|
@volatile private var logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER) extends Logging with AutoCloseable {
|
||||||
|
|
||||||
|
@ -190,40 +182,26 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize topic ID information for the log by maintaining the partition metadata file and setting the in-memory _topicId.
|
* Initialize topic ID information for the log by maintaining the partition metadata file and setting the in-memory _topicId.
|
||||||
* Delete partition metadata file if the version does not support topic IDs.
|
|
||||||
* Set _topicId based on a few scenarios:
|
* Set _topicId based on a few scenarios:
|
||||||
* - Recover topic ID if present and topic IDs are supported. Ensure we do not try to assign a provided topicId that is inconsistent
|
* - Recover topic ID if present. Ensure we do not try to assign a provided topicId that is inconsistent
|
||||||
* with the ID on file.
|
* with the ID on file.
|
||||||
* - If we were provided a topic ID when creating the log, partition metadata files are supported, and one does not yet exist
|
* - If we were provided a topic ID when creating the log and one does not yet exist
|
||||||
* set _topicId and write to the partition metadata file.
|
* set _topicId and write to the partition metadata file.
|
||||||
* - Otherwise set _topicId to None
|
|
||||||
*/
|
*/
|
||||||
private def initializeTopicId(): Unit = {
|
private def initializeTopicId(): Unit = {
|
||||||
val partMetadataFile = partitionMetadataFile.getOrElse(
|
val partMetadataFile = partitionMetadataFile.getOrElse(
|
||||||
throw new KafkaException("The partitionMetadataFile should have been initialized"))
|
throw new KafkaException("The partitionMetadataFile should have been initialized"))
|
||||||
|
|
||||||
if (partMetadataFile.exists()) {
|
if (partMetadataFile.exists()) {
|
||||||
if (keepPartitionMetadataFile) {
|
|
||||||
val fileTopicId = partMetadataFile.read().topicId
|
val fileTopicId = partMetadataFile.read().topicId
|
||||||
if (_topicId.isDefined && !_topicId.contains(fileTopicId))
|
if (_topicId.isDefined && !_topicId.contains(fileTopicId))
|
||||||
throw new InconsistentTopicIdException(s"Tried to assign topic ID $topicId to log for topic partition $topicPartition," +
|
throw new InconsistentTopicIdException(s"Tried to assign topic ID $topicId to log for topic partition $topicPartition," +
|
||||||
s"but log already contained topic ID $fileTopicId")
|
s"but log already contained topic ID $fileTopicId")
|
||||||
|
|
||||||
_topicId = Some(fileTopicId)
|
_topicId = Some(fileTopicId)
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
try partMetadataFile.delete()
|
|
||||||
catch {
|
|
||||||
case e: IOException =>
|
|
||||||
error(s"Error while trying to delete partition metadata file $partMetadataFile", e)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else if (keepPartitionMetadataFile) {
|
|
||||||
_topicId.foreach(partMetadataFile.record)
|
_topicId.foreach(partMetadataFile.record)
|
||||||
scheduler.scheduleOnce("flush-metadata-file", () => maybeFlushMetadataFile())
|
scheduler.scheduleOnce("flush-metadata-file", () => maybeFlushMetadataFile())
|
||||||
} else {
|
|
||||||
// We want to keep the file and the in-memory topic ID in sync.
|
|
||||||
_topicId = None
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -493,7 +471,6 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
}
|
}
|
||||||
|
|
||||||
case None =>
|
case None =>
|
||||||
if (keepPartitionMetadataFile) {
|
|
||||||
_topicId = Some(topicId)
|
_topicId = Some(topicId)
|
||||||
partitionMetadataFile match {
|
partitionMetadataFile match {
|
||||||
case Some(partMetadataFile) =>
|
case Some(partMetadataFile) =>
|
||||||
|
@ -506,7 +483,6 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
private def reinitializeLeaderEpochCache(): Unit = lock synchronized {
|
private def reinitializeLeaderEpochCache(): Unit = lock synchronized {
|
||||||
leaderEpochCache = UnifiedLog.createLeaderEpochCache(
|
leaderEpochCache = UnifiedLog.createLeaderEpochCache(
|
||||||
|
@ -1989,7 +1965,6 @@ object UnifiedLog extends Logging {
|
||||||
logDirFailureChannel: LogDirFailureChannel,
|
logDirFailureChannel: LogDirFailureChannel,
|
||||||
lastShutdownClean: Boolean = true,
|
lastShutdownClean: Boolean = true,
|
||||||
topicId: Option[Uuid],
|
topicId: Option[Uuid],
|
||||||
keepPartitionMetadataFile: Boolean,
|
|
||||||
numRemainingSegments: ConcurrentMap[String, Integer] = new ConcurrentHashMap[String, Integer],
|
numRemainingSegments: ConcurrentMap[String, Integer] = new ConcurrentHashMap[String, Integer],
|
||||||
remoteStorageSystemEnable: Boolean = false,
|
remoteStorageSystemEnable: Boolean = false,
|
||||||
logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER): UnifiedLog = {
|
logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER): UnifiedLog = {
|
||||||
|
@ -2034,7 +2009,6 @@ object UnifiedLog extends Logging {
|
||||||
leaderEpochCache,
|
leaderEpochCache,
|
||||||
producerStateManager,
|
producerStateManager,
|
||||||
topicId,
|
topicId,
|
||||||
keepPartitionMetadataFile,
|
|
||||||
remoteStorageSystemEnable,
|
remoteStorageSystemEnable,
|
||||||
logOffsetsListener)
|
logOffsetsListener)
|
||||||
}
|
}
|
||||||
|
|
|
@ -620,8 +620,7 @@ object KafkaMetadataLog extends Logging {
|
||||||
producerIdExpirationCheckIntervalMs = Int.MaxValue,
|
producerIdExpirationCheckIntervalMs = Int.MaxValue,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(5),
|
logDirFailureChannel = new LogDirFailureChannel(5),
|
||||||
lastShutdownClean = false,
|
lastShutdownClean = false,
|
||||||
topicId = Some(topicId),
|
topicId = Some(topicId)
|
||||||
keepPartitionMetadataFile = true
|
|
||||||
)
|
)
|
||||||
|
|
||||||
val metadataLog = new KafkaMetadataLog(
|
val metadataLog = new KafkaMetadataLog(
|
||||||
|
|
|
@ -216,8 +216,7 @@ class BrokerServer(
|
||||||
kafkaScheduler,
|
kafkaScheduler,
|
||||||
time,
|
time,
|
||||||
brokerTopicStats,
|
brokerTopicStats,
|
||||||
logDirFailureChannel,
|
logDirFailureChannel)
|
||||||
keepPartitionMetadataFile = true)
|
|
||||||
|
|
||||||
remoteLogManagerOpt = createRemoteLogManager()
|
remoteLogManagerOpt = createRemoteLogManager()
|
||||||
|
|
||||||
|
|
|
@ -452,8 +452,7 @@ class PartitionLockTest extends Logging {
|
||||||
log.producerIdExpirationCheckIntervalMs,
|
log.producerIdExpirationCheckIntervalMs,
|
||||||
leaderEpochCache,
|
leaderEpochCache,
|
||||||
producerStateManager,
|
producerStateManager,
|
||||||
_topicId = None,
|
_topicId = None) {
|
||||||
keepPartitionMetadataFile = true) {
|
|
||||||
|
|
||||||
override def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, origin: AppendOrigin,
|
override def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, origin: AppendOrigin,
|
||||||
requestLocal: RequestLocal, verificationGuard: VerificationGuard): LogAppendInfo = {
|
requestLocal: RequestLocal, verificationGuard: VerificationGuard): LogAppendInfo = {
|
||||||
|
|
|
@ -3622,8 +3622,7 @@ class PartitionTest extends AbstractPartitionTest {
|
||||||
log.producerIdExpirationCheckIntervalMs,
|
log.producerIdExpirationCheckIntervalMs,
|
||||||
leaderEpochCache,
|
leaderEpochCache,
|
||||||
producerStateManager,
|
producerStateManager,
|
||||||
_topicId = None,
|
_topicId = None) {
|
||||||
keepPartitionMetadataFile = true) {
|
|
||||||
|
|
||||||
override def appendAsFollower(records: MemoryRecords): LogAppendInfo = {
|
override def appendAsFollower(records: MemoryRecords): LogAppendInfo = {
|
||||||
appendSemaphore.acquire()
|
appendSemaphore.acquire()
|
||||||
|
|
|
@ -117,8 +117,7 @@ abstract class AbstractLogCleanerIntegrationTest {
|
||||||
producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false),
|
producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false),
|
||||||
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(10),
|
logDirFailureChannel = new LogDirFailureChannel(10),
|
||||||
topicId = None,
|
topicId = None)
|
||||||
keepPartitionMetadataFile = true)
|
|
||||||
logMap.put(partition, log)
|
logMap.put(partition, log)
|
||||||
this.logs += log
|
this.logs += log
|
||||||
}
|
}
|
||||||
|
|
|
@ -69,8 +69,7 @@ class BrokerCompressionTest {
|
||||||
producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false),
|
producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false),
|
||||||
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(10),
|
logDirFailureChannel = new LogDirFailureChannel(10),
|
||||||
topicId = None,
|
topicId = None
|
||||||
keepPartitionMetadataFile = true
|
|
||||||
)
|
)
|
||||||
|
|
||||||
/* append two messages */
|
/* append two messages */
|
||||||
|
|
|
@ -133,7 +133,7 @@ class LogCleanerManagerTest extends Logging {
|
||||||
// the exception should be caught and the partition that caused it marked as uncleanable
|
// the exception should be caught and the partition that caused it marked as uncleanable
|
||||||
class LogMock extends UnifiedLog(offsets.logStartOffset, localLog, new BrokerTopicStats,
|
class LogMock extends UnifiedLog(offsets.logStartOffset, localLog, new BrokerTopicStats,
|
||||||
producerIdExpirationCheckIntervalMs, leaderEpochCache,
|
producerIdExpirationCheckIntervalMs, leaderEpochCache,
|
||||||
producerStateManager, _topicId = None, keepPartitionMetadataFile = true) {
|
producerStateManager, _topicId = None) {
|
||||||
// Throw an error in getFirstBatchTimestampForSegments since it is called in grabFilthiestLog()
|
// Throw an error in getFirstBatchTimestampForSegments since it is called in grabFilthiestLog()
|
||||||
override def getFirstBatchTimestampForSegments(segments: util.Collection[LogSegment]): util.Collection[java.lang.Long] =
|
override def getFirstBatchTimestampForSegments(segments: util.Collection[LogSegment]): util.Collection[java.lang.Long] =
|
||||||
throw new IllegalStateException("Error!")
|
throw new IllegalStateException("Error!")
|
||||||
|
@ -821,8 +821,7 @@ class LogCleanerManagerTest extends Logging {
|
||||||
producerStateManagerConfig = producerStateManagerConfig,
|
producerStateManagerConfig = producerStateManagerConfig,
|
||||||
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(10),
|
logDirFailureChannel = new LogDirFailureChannel(10),
|
||||||
topicId = None,
|
topicId = None)
|
||||||
keepPartitionMetadataFile = true)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = {
|
private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = {
|
||||||
|
@ -875,8 +874,7 @@ class LogCleanerManagerTest extends Logging {
|
||||||
producerStateManagerConfig = producerStateManagerConfig,
|
producerStateManagerConfig = producerStateManagerConfig,
|
||||||
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(10),
|
logDirFailureChannel = new LogDirFailureChannel(10),
|
||||||
topicId = None,
|
topicId = None
|
||||||
keepPartitionMetadataFile = true
|
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -216,8 +216,7 @@ class LogCleanerTest extends Logging {
|
||||||
producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs,
|
producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs,
|
||||||
leaderEpochCache = leaderEpochCache,
|
leaderEpochCache = leaderEpochCache,
|
||||||
producerStateManager = producerStateManager,
|
producerStateManager = producerStateManager,
|
||||||
_topicId = None,
|
_topicId = None) {
|
||||||
keepPartitionMetadataFile = true) {
|
|
||||||
override def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment]): Unit = {
|
override def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment]): Unit = {
|
||||||
deleteStartLatch.countDown()
|
deleteStartLatch.countDown()
|
||||||
if (!deleteCompleteLatch.await(5000, TimeUnit.MILLISECONDS)) {
|
if (!deleteCompleteLatch.await(5000, TimeUnit.MILLISECONDS)) {
|
||||||
|
@ -2093,8 +2092,7 @@ class LogCleanerTest extends Logging {
|
||||||
producerStateManagerConfig = producerStateManagerConfig,
|
producerStateManagerConfig = producerStateManagerConfig,
|
||||||
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(10),
|
logDirFailureChannel = new LogDirFailureChannel(10),
|
||||||
topicId = None,
|
topicId = None
|
||||||
keepPartitionMetadataFile = true
|
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -156,8 +156,7 @@ class LogConcurrencyTest {
|
||||||
producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false),
|
producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false),
|
||||||
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(10),
|
logDirFailureChannel = new LogDirFailureChannel(10),
|
||||||
topicId = None,
|
topicId = None
|
||||||
keepPartitionMetadataFile = true
|
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -127,7 +127,6 @@ class LogLoaderTest {
|
||||||
brokerTopicStats = new BrokerTopicStats(),
|
brokerTopicStats = new BrokerTopicStats(),
|
||||||
logDirFailureChannel = logDirFailureChannel,
|
logDirFailureChannel = logDirFailureChannel,
|
||||||
time = time,
|
time = time,
|
||||||
keepPartitionMetadataFile = true,
|
|
||||||
remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled(),
|
remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled(),
|
||||||
initialTaskDelayMs = config.logInitialTaskDelayMs) {
|
initialTaskDelayMs = config.logInitialTaskDelayMs) {
|
||||||
|
|
||||||
|
@ -324,7 +323,7 @@ class LogLoaderTest {
|
||||||
logDirFailureChannel)
|
logDirFailureChannel)
|
||||||
new UnifiedLog(offsets.logStartOffset, localLog, brokerTopicStats,
|
new UnifiedLog(offsets.logStartOffset, localLog, brokerTopicStats,
|
||||||
producerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager,
|
producerIdExpirationCheckIntervalMs, leaderEpochCache, producerStateManager,
|
||||||
None, keepPartitionMetadataFile = true)
|
None)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Retain snapshots for the last 2 segments
|
// Retain snapshots for the last 2 segments
|
||||||
|
@ -447,8 +446,7 @@ class LogLoaderTest {
|
||||||
producerIdExpirationCheckIntervalMs = 30000,
|
producerIdExpirationCheckIntervalMs = 30000,
|
||||||
leaderEpochCache = leaderEpochCache,
|
leaderEpochCache = leaderEpochCache,
|
||||||
producerStateManager = stateManager,
|
producerStateManager = stateManager,
|
||||||
_topicId = None,
|
_topicId = None)
|
||||||
keepPartitionMetadataFile = true)
|
|
||||||
|
|
||||||
verify(stateManager).updateMapEndOffset(0L)
|
verify(stateManager).updateMapEndOffset(0L)
|
||||||
verify(stateManager).removeStraySnapshots(any())
|
verify(stateManager).removeStraySnapshots(any())
|
||||||
|
@ -557,8 +555,7 @@ class LogLoaderTest {
|
||||||
producerIdExpirationCheckIntervalMs = 30000,
|
producerIdExpirationCheckIntervalMs = 30000,
|
||||||
leaderEpochCache = leaderEpochCache,
|
leaderEpochCache = leaderEpochCache,
|
||||||
producerStateManager = stateManager,
|
producerStateManager = stateManager,
|
||||||
_topicId = None,
|
_topicId = None)
|
||||||
keepPartitionMetadataFile = true)
|
|
||||||
|
|
||||||
verify(stateManager).removeStraySnapshots(any[java.util.List[java.lang.Long]])
|
verify(stateManager).removeStraySnapshots(any[java.util.List[java.lang.Long]])
|
||||||
verify(stateManager, times(2)).updateMapEndOffset(0L)
|
verify(stateManager, times(2)).updateMapEndOffset(0L)
|
||||||
|
|
|
@ -975,7 +975,6 @@ class LogManagerTest {
|
||||||
// not clean shutdown
|
// not clean shutdown
|
||||||
lastShutdownClean = false,
|
lastShutdownClean = false,
|
||||||
topicId = None,
|
topicId = None,
|
||||||
keepPartitionMetadataFile = false,
|
|
||||||
// pass mock map for verification later
|
// pass mock map for verification later
|
||||||
numRemainingSegments = mockMap)
|
numRemainingSegments = mockMap)
|
||||||
|
|
||||||
|
@ -1383,7 +1382,6 @@ class LogManagerTest {
|
||||||
time = Time.SYSTEM,
|
time = Time.SYSTEM,
|
||||||
brokerTopicStats = new BrokerTopicStats,
|
brokerTopicStats = new BrokerTopicStats,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(1),
|
logDirFailureChannel = new LogDirFailureChannel(1),
|
||||||
keepPartitionMetadataFile = true,
|
|
||||||
interBrokerProtocolVersion = MetadataVersion.latestTesting,
|
interBrokerProtocolVersion = MetadataVersion.latestTesting,
|
||||||
remoteStorageSystemEnable = false,
|
remoteStorageSystemEnable = false,
|
||||||
initialTaskDelayMs = 0)
|
initialTaskDelayMs = 0)
|
||||||
|
|
|
@ -103,7 +103,6 @@ object LogTestUtils {
|
||||||
producerIdExpirationCheckIntervalMs: Int = TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs: Int = TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT,
|
||||||
lastShutdownClean: Boolean = true,
|
lastShutdownClean: Boolean = true,
|
||||||
topicId: Option[Uuid] = None,
|
topicId: Option[Uuid] = None,
|
||||||
keepPartitionMetadataFile: Boolean = true,
|
|
||||||
numRemainingSegments: ConcurrentMap[String, Integer] = new ConcurrentHashMap[String, Integer],
|
numRemainingSegments: ConcurrentMap[String, Integer] = new ConcurrentHashMap[String, Integer],
|
||||||
remoteStorageSystemEnable: Boolean = false,
|
remoteStorageSystemEnable: Boolean = false,
|
||||||
remoteLogManager: Option[RemoteLogManager] = None,
|
remoteLogManager: Option[RemoteLogManager] = None,
|
||||||
|
@ -122,7 +121,6 @@ object LogTestUtils {
|
||||||
logDirFailureChannel = new LogDirFailureChannel(10),
|
logDirFailureChannel = new LogDirFailureChannel(10),
|
||||||
lastShutdownClean = lastShutdownClean,
|
lastShutdownClean = lastShutdownClean,
|
||||||
topicId = topicId,
|
topicId = topicId,
|
||||||
keepPartitionMetadataFile = keepPartitionMetadataFile,
|
|
||||||
numRemainingSegments = numRemainingSegments,
|
numRemainingSegments = numRemainingSegments,
|
||||||
remoteStorageSystemEnable = remoteStorageSystemEnable,
|
remoteStorageSystemEnable = remoteStorageSystemEnable,
|
||||||
logOffsetsListener = logOffsetsListener
|
logOffsetsListener = logOffsetsListener
|
||||||
|
|
|
@ -1964,26 +1964,6 @@ class UnifiedLogTest {
|
||||||
log.close()
|
log.close()
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
def testNoOpWhenKeepPartitionMetadataFileIsFalse(): Unit = {
|
|
||||||
val logConfig = LogTestUtils.createLogConfig()
|
|
||||||
val log = createLog(logDir, logConfig, keepPartitionMetadataFile = false)
|
|
||||||
|
|
||||||
val topicId = Uuid.randomUuid()
|
|
||||||
log.assignTopicId(topicId)
|
|
||||||
// We should not write to this file or set the topic ID
|
|
||||||
assertFalse(log.partitionMetadataFile.get.exists())
|
|
||||||
assertEquals(None, log.topicId)
|
|
||||||
log.close()
|
|
||||||
|
|
||||||
val log2 = createLog(logDir, logConfig, topicId = Some(Uuid.randomUuid()), keepPartitionMetadataFile = false)
|
|
||||||
|
|
||||||
// We should not write to this file or set the topic ID
|
|
||||||
assertFalse(log2.partitionMetadataFile.get.exists())
|
|
||||||
assertEquals(None, log2.topicId)
|
|
||||||
log2.close()
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testLogFailsWhenInconsistentTopicIdSet(): Unit = {
|
def testLogFailsWhenInconsistentTopicIdSet(): Unit = {
|
||||||
val logConfig = LogTestUtils.createLogConfig()
|
val logConfig = LogTestUtils.createLogConfig()
|
||||||
|
@ -4499,13 +4479,12 @@ class UnifiedLogTest {
|
||||||
producerIdExpirationCheckIntervalMs: Int = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs: Int = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
||||||
lastShutdownClean: Boolean = true,
|
lastShutdownClean: Boolean = true,
|
||||||
topicId: Option[Uuid] = None,
|
topicId: Option[Uuid] = None,
|
||||||
keepPartitionMetadataFile: Boolean = true,
|
|
||||||
remoteStorageSystemEnable: Boolean = false,
|
remoteStorageSystemEnable: Boolean = false,
|
||||||
remoteLogManager: Option[RemoteLogManager] = None,
|
remoteLogManager: Option[RemoteLogManager] = None,
|
||||||
logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER): UnifiedLog = {
|
logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER): UnifiedLog = {
|
||||||
val log = LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint,
|
val log = LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint,
|
||||||
maxTransactionTimeoutMs, producerStateManagerConfig, producerIdExpirationCheckIntervalMs,
|
maxTransactionTimeoutMs, producerStateManagerConfig, producerIdExpirationCheckIntervalMs,
|
||||||
lastShutdownClean, topicId, keepPartitionMetadataFile, new ConcurrentHashMap[String, Integer],
|
lastShutdownClean, topicId, new ConcurrentHashMap[String, Integer],
|
||||||
remoteStorageSystemEnable, remoteLogManager, logOffsetsListener)
|
remoteStorageSystemEnable, remoteLogManager, logOffsetsListener)
|
||||||
logsToClose = logsToClose :+ log
|
logsToClose = logsToClose :+ log
|
||||||
log
|
log
|
||||||
|
|
|
@ -2929,8 +2929,7 @@ class ReplicaManagerTest {
|
||||||
producerIdExpirationCheckIntervalMs = 30000,
|
producerIdExpirationCheckIntervalMs = 30000,
|
||||||
leaderEpochCache = leaderEpochCache,
|
leaderEpochCache = leaderEpochCache,
|
||||||
producerStateManager = producerStateManager,
|
producerStateManager = producerStateManager,
|
||||||
_topicId = topicId,
|
_topicId = topicId) {
|
||||||
keepPartitionMetadataFile = true) {
|
|
||||||
|
|
||||||
override def endOffsetForEpoch(leaderEpoch: Int): Option[OffsetAndEpoch] = {
|
override def endOffsetForEpoch(leaderEpoch: Int): Option[OffsetAndEpoch] = {
|
||||||
assertEquals(leaderEpoch, leaderEpochFromLeader)
|
assertEquals(leaderEpoch, leaderEpochFromLeader)
|
||||||
|
|
|
@ -100,8 +100,7 @@ class DumpLogSegmentsTest {
|
||||||
producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false),
|
producerStateManagerConfig = new ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT, false),
|
||||||
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(10),
|
logDirFailureChannel = new LogDirFailureChannel(10),
|
||||||
topicId = None,
|
topicId = None
|
||||||
keepPartitionMetadataFile = true
|
|
||||||
)
|
)
|
||||||
log
|
log
|
||||||
}
|
}
|
||||||
|
|
|
@ -164,7 +164,7 @@ class SchedulerTest {
|
||||||
localLog = localLog,
|
localLog = localLog,
|
||||||
brokerTopicStats, producerIdExpirationCheckIntervalMs,
|
brokerTopicStats, producerIdExpirationCheckIntervalMs,
|
||||||
leaderEpochCache, producerStateManager,
|
leaderEpochCache, producerStateManager,
|
||||||
_topicId = None, keepPartitionMetadataFile = true)
|
_topicId = None)
|
||||||
assertTrue(scheduler.taskRunning(log.producerExpireCheck))
|
assertTrue(scheduler.taskRunning(log.producerExpireCheck))
|
||||||
log.close()
|
log.close()
|
||||||
assertFalse(scheduler.taskRunning(log.producerExpireCheck))
|
assertFalse(scheduler.taskRunning(log.producerExpireCheck))
|
||||||
|
|
|
@ -988,7 +988,6 @@ object TestUtils extends Logging {
|
||||||
time = time,
|
time = time,
|
||||||
brokerTopicStats = new BrokerTopicStats,
|
brokerTopicStats = new BrokerTopicStats,
|
||||||
logDirFailureChannel = new LogDirFailureChannel(logDirs.size),
|
logDirFailureChannel = new LogDirFailureChannel(logDirs.size),
|
||||||
keepPartitionMetadataFile = true,
|
|
||||||
interBrokerProtocolVersion = interBrokerProtocolVersion,
|
interBrokerProtocolVersion = interBrokerProtocolVersion,
|
||||||
remoteStorageSystemEnable = remoteStorageSystemEnable,
|
remoteStorageSystemEnable = remoteStorageSystemEnable,
|
||||||
initialTaskDelayMs = initialTaskDelayMs)
|
initialTaskDelayMs = initialTaskDelayMs)
|
||||||
|
|
|
@ -146,7 +146,6 @@ public class ReplicaFetcherThreadBenchmark {
|
||||||
setBrokerTopicStats(brokerTopicStats).
|
setBrokerTopicStats(brokerTopicStats).
|
||||||
setLogDirFailureChannel(logDirFailureChannel).
|
setLogDirFailureChannel(logDirFailureChannel).
|
||||||
setTime(Time.SYSTEM).
|
setTime(Time.SYSTEM).
|
||||||
setKeepPartitionMetadataFile(true).
|
|
||||||
build();
|
build();
|
||||||
|
|
||||||
replicaManager = new ReplicaManagerBuilder().
|
replicaManager = new ReplicaManagerBuilder().
|
||||||
|
|
|
@ -78,7 +78,6 @@ public class StressTestLog {
|
||||||
new LogDirFailureChannel(10),
|
new LogDirFailureChannel(10),
|
||||||
true,
|
true,
|
||||||
Option.empty(),
|
Option.empty(),
|
||||||
true,
|
|
||||||
new ConcurrentHashMap<>(),
|
new ConcurrentHashMap<>(),
|
||||||
false,
|
false,
|
||||||
LogOffsetsListener.NO_OP_OFFSETS_LISTENER
|
LogOffsetsListener.NO_OP_OFFSETS_LISTENER
|
||||||
|
|
|
@ -314,7 +314,6 @@ public class TestLinearWriteSpeed {
|
||||||
new LogDirFailureChannel(10),
|
new LogDirFailureChannel(10),
|
||||||
true,
|
true,
|
||||||
Option.empty(),
|
Option.empty(),
|
||||||
true,
|
|
||||||
new CopyOnWriteMap<>(),
|
new CopyOnWriteMap<>(),
|
||||||
false,
|
false,
|
||||||
LogOffsetsListener.NO_OP_OFFSETS_LISTENER
|
LogOffsetsListener.NO_OP_OFFSETS_LISTENER
|
||||||
|
|
|
@ -115,7 +115,7 @@ public class PartitionMakeFollowerBenchmark {
|
||||||
setScheduler(scheduler).
|
setScheduler(scheduler).
|
||||||
setBrokerTopicStats(brokerTopicStats).
|
setBrokerTopicStats(brokerTopicStats).
|
||||||
setLogDirFailureChannel(logDirFailureChannel).
|
setLogDirFailureChannel(logDirFailureChannel).
|
||||||
setTime(Time.SYSTEM).setKeepPartitionMetadataFile(true).
|
setTime(Time.SYSTEM).
|
||||||
build();
|
build();
|
||||||
|
|
||||||
TopicPartition tp = new TopicPartition("topic", 0);
|
TopicPartition tp = new TopicPartition("topic", 0);
|
||||||
|
|
|
@ -106,7 +106,6 @@ public class UpdateFollowerFetchStateBenchmark {
|
||||||
setBrokerTopicStats(brokerTopicStats).
|
setBrokerTopicStats(brokerTopicStats).
|
||||||
setLogDirFailureChannel(logDirFailureChannel).
|
setLogDirFailureChannel(logDirFailureChannel).
|
||||||
setTime(Time.SYSTEM).
|
setTime(Time.SYSTEM).
|
||||||
setKeepPartitionMetadataFile(true).
|
|
||||||
build();
|
build();
|
||||||
OffsetCheckpoints offsetCheckpoints = Mockito.mock(OffsetCheckpoints.class);
|
OffsetCheckpoints offsetCheckpoints = Mockito.mock(OffsetCheckpoints.class);
|
||||||
Mockito.when(offsetCheckpoints.fetch(logDir.getAbsolutePath(), topicPartition)).thenReturn(Optional.of(0L));
|
Mockito.when(offsetCheckpoints.fetch(logDir.getAbsolutePath(), topicPartition)).thenReturn(Optional.of(0L));
|
||||||
|
|
|
@ -141,7 +141,6 @@ public class PartitionCreationBench {
|
||||||
setBrokerTopicStats(brokerTopicStats).
|
setBrokerTopicStats(brokerTopicStats).
|
||||||
setLogDirFailureChannel(failureChannel).
|
setLogDirFailureChannel(failureChannel).
|
||||||
setTime(Time.SYSTEM).
|
setTime(Time.SYSTEM).
|
||||||
setKeepPartitionMetadataFile(true).
|
|
||||||
build();
|
build();
|
||||||
scheduler.startup();
|
scheduler.startup();
|
||||||
this.quotaManagers = QuotaFactory.instantiate(this.brokerProperties, this.metrics, this.time, "");
|
this.quotaManagers = QuotaFactory.instantiate(this.brokerProperties, this.metrics, this.time, "");
|
||||||
|
|
Loading…
Reference in New Issue