Skip to content

[WIP]KAFKA-19080 The constraint on segment.ms is not enforced at topic level #19371

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 47 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 29 commits
Commits
Show all changes
47 commits
Select commit Hold shift + click to select a range
ef72eee
fix the topic level segment.bytes error
m1a2st Apr 4, 2025
36e59c8
use spotlessApply
m1a2st Apr 4, 2025
7a29f51
update test default value
m1a2st Apr 4, 2025
1d66fb0
wip
m1a2st Apr 4, 2025
60c01e9
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 9, 2025
9398bae
revert test change
m1a2st Apr 9, 2025
0f766c5
add new constructor for test
m1a2st Apr 9, 2025
a0342ed
add escape annotation
m1a2st Apr 9, 2025
f707465
fix some test
m1a2st Apr 9, 2025
768c8ed
Revert "fix some test"
m1a2st Apr 10, 2025
640ca6b
Revert "add escape annotation"
m1a2st Apr 10, 2025
2bfd478
Revert "add new constructor for test"
m1a2st Apr 10, 2025
51da598
fix all test
m1a2st Apr 10, 2025
8c39276
fix some test
m1a2st Apr 10, 2025
eefa646
fix MetadataLog Test
m1a2st Apr 10, 2025
e2adfbe
fix MetadataLog Test
m1a2st Apr 10, 2025
3d8e122
fix some test
m1a2st Apr 10, 2025
303ef39
fix fail tests
m1a2st Apr 11, 2025
3f1b757
Merge remote-tracking branch 'origin/KAFKA-19080' into KAFKA-19080
m1a2st Apr 11, 2025
71dbb89
fix config def error
m1a2st Apr 11, 2025
4c62f11
fix fail test
m1a2st Apr 11, 2025
c847bbc
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 11, 2025
ad36e75
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 12, 2025
5e4fe46
Update LogCleaner file
m1a2st Apr 12, 2025
9b9f469
change segmentSize modifier
m1a2st Apr 12, 2025
4e7a080
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 14, 2025
8b7d366
addressed by comments
m1a2st Apr 14, 2025
5bbdf1f
addressed by comments
m1a2st Apr 14, 2025
2a3db85
update the test
m1a2st Apr 14, 2025
919365a
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 15, 2025
a332072
remove unused import
m1a2st Apr 15, 2025
b774d92
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 20, 2025
6876739
move other internal config
m1a2st Apr 20, 2025
34a794f
update KafkaMetadataLog apply flow
m1a2st Apr 20, 2025
c10c9cb
fix compile error
m1a2st Apr 20, 2025
ea981e3
fix fail test
m1a2st Apr 21, 2025
934ac37
fix fail test
m1a2st Apr 21, 2025
77898ed
revert unused change
m1a2st Apr 21, 2025
1756fee
revert unused change
m1a2st Apr 21, 2025
0bfec85
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 22, 2025
1a3f737
temp
m1a2st Apr 22, 2025
5761c62
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 24, 2025
06131bf
Merge branch 'trunk' into KAFKA-19080
m1a2st Apr 24, 2025
7483a5d
completed the feature
m1a2st Apr 25, 2025
048e052
fix fail test
m1a2st Apr 25, 2025
76bd287
fix fail test
m1a2st Apr 25, 2025
ce644ae
addressed by comments
m1a2st Apr 27, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
118 changes: 87 additions & 31 deletions core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
Original file line number Diff line number Diff line change
Expand Up @@ -575,6 +575,7 @@ final class KafkaMetadataLog private (
}

object KafkaMetadataLog extends Logging {

def apply(
topicPartition: TopicPartition,
topicId: Uuid,
Expand All @@ -583,35 +584,72 @@ object KafkaMetadataLog extends Logging {
scheduler: Scheduler,
config: MetadataLogConfig
): KafkaMetadataLog = {
val props = new Properties()
props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString)
props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString)
props.setProperty(TopicConfig.SEGMENT_MS_CONFIG, config.logSegmentMillis.toString)
props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString)

// Disable time and byte retention when deleting segments
props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1")
props.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "-1")
val props: Properties = settingLogProperties(config)
LogConfig.validate(props)
val defaultLogConfig = new LogConfig(props)

if (config.logSegmentBytes < config.logSegmentMinBytes) {
throw new InvalidConfigurationException(
s"Cannot set ${KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG} below ${config.logSegmentMinBytes}: ${config.logSegmentBytes}"
)
} else if (defaultLogConfig.retentionMs >= 0) {
throw new InvalidConfigurationException(
s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${defaultLogConfig.retentionMs}."
)
} else if (defaultLogConfig.retentionSize >= 0) {
throw new InvalidConfigurationException(
s"Cannot set ${TopicConfig.RETENTION_BYTES_CONFIG} above -1: ${defaultLogConfig.retentionSize}."
)
validateConfig(config, defaultLogConfig)

val metadataLog: KafkaMetadataLog = createKafkaMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, defaultLogConfig)

printWarningMessage(config, metadataLog)

// When recovering, truncate fully if the latest snapshot is after the log end offset. This can happen to a follower
// when the follower crashes after downloading a snapshot from the leader but before it could truncate the log fully.
metadataLog.truncateToLatestSnapshot()

metadataLog
}

private def printWarningMessage(config: MetadataLogConfig, metadataLog: KafkaMetadataLog): Unit = {
// Print a warning if users have overridden the internal config
if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) {
metadataLog.error(s"Overriding ${KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG} is only supported for testing. Setting " +
s"this value too low may lead to an inability to write batches of metadata records.")
}
}

// visible for testing
def internalApply(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of having an internalApply, could we just use the existing apply and add INTERNAL_SEGMENT_BYTES_CONFIG to MetadataLogConfig?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 to move the internal config to MetadataLogConfig, and it would be better to wait #19465 extracting the metadata-related configs from other class to MetadataLogConfig

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I just realized that metadata log uses a different approach to allow tests to use a smaller segment bytes than allowed in production. That approach defines the original segment byte config with a small minimal requirement, but adds METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG to enforce the actual minimal requirement in production. This new config could be changed in tests to allow for smaller minimal bytes. The benefit of this approach is that it allows the existing config to be used directly to set a smaller value for tests. The downside is that the doc for min value is inaccurate and the validation is done through a customized logic.

It would be useful to pick the same strategy between metadata log and regular log. The metadata log approach seems slightly better since it's less intrusive. We could fix the inaccurate min value description for production somehow.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Excuse me, the strategy used by metadata log is to add a "internal" config (METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG) to change the (metadata) segment size in testing, and that is what we want to address in this PR - we add a "internal" config for regular log, and so the test can use the "smaller" segment size.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am just saying that we now have two different ways to achieve the same goal. In the metadata log approach, you set the desired value through the original config, which is segment.bytes. You then set an internal config to change the min constraint.

The approach in this PR is to set the desired value through a different internal config.

It would be useful to choose same approach for both the metadata log and the regular log.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Personally, I prefer the approach of adding "internal".xxx config as it provide better user experience for public configs, allowing users to see the "correct" min value. Additionally, we can remove the customized logic of validation.

In short, I suggest to add following changes to this PR.

  1. remove METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG
  2. remove MetadataLogConfig#logSegmentMinBytes
  3. add internal.metadata.log.segment.bytes
  4. customize MetadataLogConfig#logSegmentBytes as following code
    public int logSegmentBytes() {
        if (internalSogSegmentBytes != null) return internalSogSegmentBytes;
        return logSegmentBytes;
    }

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for @chia7712, @junrao comments, addressed it :)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, why do we need internalApply()? MetadataLogConfig has both the product and internal segment bytes configs and we could just pass both into LogConfig in apply(), right?

topicPartition: TopicPartition,
topicId: Uuid,
dataDir: File,
time: Time,
scheduler: Scheduler,
config: MetadataLogConfig
): KafkaMetadataLog = {
val props: Properties = settingLogProperties(config)
props.remove(TopicConfig.SEGMENT_BYTES_CONFIG);
props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString)
LogConfig.validate(props)
val defaultLogConfig = new LogConfig(props)

validateConfig(config, defaultLogConfig)

val metadataLog: KafkaMetadataLog = createKafkaMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, defaultLogConfig)

// Print a warning if users have overridden the internal config
printWarningMessage(config, metadataLog)

// When recovering, truncate fully if the latest snapshot is after the log end offset. This can happen to a follower
// when the follower crashes after downloading a snapshot from the leader but before it could truncate the log fully.
metadataLog.truncateToLatestSnapshot()

metadataLog
}

private def createKafkaMetadataLog(
topicPartition: TopicPartition,
topicId: Uuid,
dataDir: File,
time: Time,
scheduler: Scheduler,
metadataLogConfig: MetadataLogConfig,
logConfig: LogConfig
): KafkaMetadataLog = {
val log = UnifiedLog.create(
dataDir,
defaultLogConfig,
logConfig,
0L,
0L,
scheduler,
Expand All @@ -631,20 +669,38 @@ object KafkaMetadataLog extends Logging {
scheduler,
recoverSnapshots(log),
topicPartition,
config
metadataLogConfig
)
metadataLog
}

// Print a warning if users have overridden the internal config
if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) {
metadataLog.error(s"Overriding ${KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG} is only supported for testing. Setting " +
s"this value too low may lead to an inability to write batches of metadata records.")
private def validateConfig(config: MetadataLogConfig, defaultLogConfig: LogConfig): Unit = {
if (config.logSegmentBytes < config.logSegmentMinBytes) {
throw new InvalidConfigurationException(
s"Cannot set ${KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG} below ${config.logSegmentMinBytes}: ${config.logSegmentBytes}"
)
} else if (defaultLogConfig.retentionMs >= 0) {
throw new InvalidConfigurationException(
s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${defaultLogConfig.retentionMs}."
)
} else if (defaultLogConfig.retentionSize >= 0) {
throw new InvalidConfigurationException(
s"Cannot set ${TopicConfig.RETENTION_BYTES_CONFIG} above -1: ${defaultLogConfig.retentionSize}."
)
}
}

// When recovering, truncate fully if the latest snapshot is after the log end offset. This can happen to a follower
// when the follower crashes after downloading a snapshot from the leader but before it could truncate the log fully.
metadataLog.truncateToLatestSnapshot()
private def settingLogProperties(config: MetadataLogConfig) = {
val props = new Properties()
props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString)
props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString)
props.setProperty(TopicConfig.SEGMENT_MS_CONFIG, config.logSegmentMillis.toString)
props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString)

metadataLog
// Disable time and byte retention when deleting segments
props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1")
props.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "-1")
props
}

private def recoverSnapshots(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1568,7 +1568,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
@MethodSource(Array("getTestGroupProtocolParametersAll"))
def testDeleteRecordsAfterCorruptRecords(groupProtocol: String): Unit = {
val config = new Properties()
config.put(TopicConfig.SEGMENT_BYTES_CONFIG, "200")
config.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "200")
createTopic(topic, numPartitions = 1, replicationFactor = 1, config)

client = createAdminClient
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import org.apache.kafka.security.authorizer.AclEntry.{WILDCARD_HOST, WILDCARD_PR
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, ServerConfigs, ServerLogConfigs}
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
import org.apache.kafka.server.authorizer.{Authorizer => JAuthorizer}
import org.apache.kafka.storage.internals.log.LogConfig
import org.apache.kafka.test.TestUtils.assertFutureThrows
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout}
Expand Down Expand Up @@ -566,7 +567,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get()

val topics = Seq(topic1, topic2)
val configsOverride = Map(TopicConfig.SEGMENT_BYTES_CONFIG -> "100000").asJava
val configsOverride = Map(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG -> "100000").asJava
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this test case is used to verify the custom topic-level config, so we can increase the value to make test pass.

val newTopics = Seq(
new NewTopic(topic1, 2, 3.toShort).configs(configsOverride),
new NewTopic(topic2, Option.empty[Integer].toJava, Option.empty[java.lang.Short].toJava).configs(configsOverride))
Expand All @@ -579,7 +580,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu
assertEquals(3, result.replicationFactor(topic1).get())
val topicConfigs = result.config(topic1).get().entries.asScala
assertTrue(topicConfigs.nonEmpty)
val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.SEGMENT_BYTES_CONFIG).get
val segmentBytesConfig = topicConfigs.find(_.name == LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG).get
assertEquals(100000, segmentBytesConfig.value.toLong)
assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source)
val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -652,7 +652,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
"Config not updated in LogManager")

val log = servers.head.logManager.getLog(new TopicPartition(topic, 0)).getOrElse(throw new IllegalStateException("Log not found"))
TestUtils.waitUntilTrue(() => log.config.segmentSize == 1048576, "Existing topic config using defaults not updated")
TestUtils.waitUntilTrue(() => {log.config.segmentSize() == 1048576}, "Existing topic config using defaults not updated")
val KafkaConfigToLogConfigName: Map[String, String] =
ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.asScala.map { case (k, v) => (v, k) }
props.asScala.foreach { case (k, v) =>
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1127,7 +1127,7 @@ object KafkaMetadataLogTest {
UnifiedLog.logDirName(KafkaRaftServer.MetadataPartition)
)

val metadataLog = KafkaMetadataLog(
val metadataLog = KafkaMetadataLog.internalApply(
KafkaRaftServer.MetadataPartition,
KafkaRaftServer.MetadataTopicId,
logDir,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ class AbstractPartitionTest {

def createLogProperties(overrides: Map[String, String]): Properties = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 512: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer)
logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1000: java.lang.Integer)
logProps.put(TopicConfig.RETENTION_MS_CONFIG, 999: java.lang.Integer)
overrides.foreach { case (k, v) => logProps.put(k, v) }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -362,7 +362,7 @@ class PartitionLockTest extends Logging {

private def createLogProperties(overrides: Map[String, String]): Properties = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 512: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer)
logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1000: java.lang.Integer)
logProps.put(TopicConfig.RETENTION_MS_CONFIG, 999: java.lang.Integer)
overrides.foreach { case (k, v) => logProps.put(k, v) }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ abstract class AbstractLogCleanerIntegrationTest {
maxCompactionLagMs: Long = defaultMaxCompactionLagMs): Properties = {
val props = new Properties()
props.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, maxMessageSize: java.lang.Integer)
props.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentSize: java.lang.Integer)
props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentSize: java.lang.Integer)
props.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 100*1024: java.lang.Integer)
props.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, deleteDelay: java.lang.Integer)
props.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT)
Expand Down
18 changes: 9 additions & 9 deletions core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ class LogCleanerManagerTest extends Logging {
val topicPartition = new TopicPartition("log", 0)
val topicPartition2 = new TopicPartition("log2", 0)
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT)
val logConfig: LogConfig = new LogConfig(logProps)
Expand Down Expand Up @@ -370,7 +370,7 @@ class LogCleanerManagerTest extends Logging {

// change cleanup policy from delete to compact
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, log.config.segmentSize: Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, log.config.segmentSize(): Integer)
logProps.put(TopicConfig.RETENTION_MS_CONFIG, log.config.retentionMs: java.lang.Long)
logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT)
logProps.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, 0: Integer)
Expand Down Expand Up @@ -548,7 +548,7 @@ class LogCleanerManagerTest extends Logging {
@Test
def testCleanableOffsetsForNone(): Unit = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)

val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))

Expand All @@ -570,7 +570,7 @@ class LogCleanerManagerTest extends Logging {
@Test
def testCleanableOffsetsActiveSegment(): Unit = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)

val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))

Expand All @@ -592,7 +592,7 @@ class LogCleanerManagerTest extends Logging {
def testCleanableOffsetsForTime(): Unit = {
val compactionLag = 60 * 60 * 1000
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, compactionLag: java.lang.Integer)

val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
Expand Down Expand Up @@ -625,7 +625,7 @@ class LogCleanerManagerTest extends Logging {
def testCleanableOffsetsForShortTime(): Unit = {
val compactionLag = 60 * 60 * 1000
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, compactionLag: java.lang.Integer)

val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
Expand Down Expand Up @@ -667,7 +667,7 @@ class LogCleanerManagerTest extends Logging {
def testUndecidedTransactionalDataNotCleanable(): Unit = {
val compactionLag = 60 * 60 * 1000
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, compactionLag: java.lang.Integer)

val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
Expand Down Expand Up @@ -711,7 +711,7 @@ class LogCleanerManagerTest extends Logging {
@Test
def testDoneCleaning(): Unit = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
while (log.numberOfSegments < 8)
log.appendAsLeader(records(log.logEndOffset.toInt, log.logEndOffset.toInt, time.milliseconds()), 0)
Expand Down Expand Up @@ -830,7 +830,7 @@ class LogCleanerManagerTest extends Logging {

private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = {
val logProps = new Properties()
logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentSize: Integer)
logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentSize: Integer)
logProps.put(TopicConfig.RETENTION_MS_CONFIG, 1: Integer)
logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, cleanupPolicy)
logProps.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, 0.05: java.lang.Double) // small for easier and clearer tests
Expand Down
Loading
Loading